From b0ca82e00633ad786da75fe888367d860f4bb59b Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 10 Feb 2026 17:02:38 +0800 Subject: [PATCH 1/7] [index] support diskann for vector index --- .../actions/build-diskann-native/action.yml | 57 ++ .github/workflows/build-diskann-native.yml | 68 ++ .../workflows/publish-diskann_snapshot.yml | 184 ++++ .../paimon-diskann-e2e-test/pom.xml | 293 ++++++ .../spark/sql/DiskAnnVectorIndexE2ETest.scala | 380 ++++++++ paimon-diskann/paimon-diskann-index/pom.xml | 108 +++ .../paimon/diskann/index/DiskAnnIndex.java | 189 ++++ .../diskann/index/DiskAnnIndexMeta.java | 106 +++ .../diskann/index/DiskAnnIndexType.java | 36 + .../index/DiskAnnScoredGlobalIndexResult.java | 48 + .../index/DiskAnnVectorGlobalIndexReader.java | 392 ++++++++ .../index/DiskAnnVectorGlobalIndexWriter.java | 260 ++++++ .../index/DiskAnnVectorGlobalIndexer.java | 53 ++ .../DiskAnnVectorGlobalIndexerFactory.java | 40 + .../index/DiskAnnVectorIndexOptions.java | 147 +++ .../diskann/index/DiskAnnVectorMetric.java | 38 + ...he.paimon.globalindex.GlobalIndexerFactory | 17 + .../DiskAnnVectorGlobalIndexScanTest.java | 334 +++++++ .../index/DiskAnnVectorGlobalIndexTest.java | 486 ++++++++++ paimon-diskann/paimon-diskann-jni/pom.xml | 100 +++ .../scripts/build-native.sh | 201 +++++ .../org/apache/paimon/diskann/DiskAnn.java | 55 ++ .../paimon/diskann/DiskAnnException.java | 31 + .../apache/paimon/diskann/DiskAnnNative.java | 91 ++ .../java/org/apache/paimon/diskann/Index.java | 165 ++++ .../org/apache/paimon/diskann/MetricType.java | 45 + .../paimon/diskann/NativeLibraryLoader.java | 256 ++++++ .../src/main/native/Cargo.lock | 682 ++++++++++++++ .../src/main/native/Cargo.toml | 29 + .../src/main/native/rust-toolchain.toml | 20 + .../src/main/native/src/lib.rs | 848 ++++++++++++++++++ .../aarch64/libpaimon_diskann_jni.dylib | Bin 0 -> 1023296 bytes .../org/apache/paimon/diskann/IndexTest.java | 443 +++++++++ paimon-diskann/pom.xml | 40 + pom.xml | 9 + 35 files changed, 6251 insertions(+) create mode 100644 .github/actions/build-diskann-native/action.yml create mode 100644 .github/workflows/build-diskann-native.yml create mode 100644 .github/workflows/publish-diskann_snapshot.yml create mode 100644 paimon-diskann/paimon-diskann-e2e-test/pom.xml create mode 100644 paimon-diskann/paimon-diskann-e2e-test/src/test/scala/org/apache/paimon/spark/sql/DiskAnnVectorIndexE2ETest.scala create mode 100644 paimon-diskann/paimon-diskann-index/pom.xml create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexMeta.java create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexType.java create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnScoredGlobalIndexResult.java create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexer.java create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexerFactory.java create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorIndexOptions.java create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorMetric.java create mode 100644 paimon-diskann/paimon-diskann-index/src/main/resources/META-INF/services/org.apache.paimon.globalindex.GlobalIndexerFactory create mode 100644 paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexScanTest.java create mode 100644 paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexTest.java create mode 100644 paimon-diskann/paimon-diskann-jni/pom.xml create mode 100755 paimon-diskann/paimon-diskann-jni/scripts/build-native.sh create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnn.java create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnException.java create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/MetricType.java create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/NativeLibraryLoader.java create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.lock create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/native/rust-toolchain.toml create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs create mode 100755 paimon-diskann/paimon-diskann-jni/src/main/resources/darwin/aarch64/libpaimon_diskann_jni.dylib create mode 100644 paimon-diskann/paimon-diskann-jni/src/test/java/org/apache/paimon/diskann/IndexTest.java create mode 100644 paimon-diskann/pom.xml diff --git a/.github/actions/build-diskann-native/action.yml b/.github/actions/build-diskann-native/action.yml new file mode 100644 index 000000000000..c350844fbf08 --- /dev/null +++ b/.github/actions/build-diskann-native/action.yml @@ -0,0 +1,57 @@ +name: 'Build DiskANN Native Library' +description: 'Build DiskANN native library (Rust JNI) for specified platform' +inputs: + platform: + description: 'Target platform (linux-amd64, linux-aarch64, darwin-aarch64)' + required: true + rust-version: + description: 'Rust toolchain version' + required: false + default: 'stable' + +runs: + using: 'composite' + steps: + - name: Install Rust toolchain + shell: bash + run: | + curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh -s -- -y --default-toolchain ${{ inputs.rust-version }} + echo "$HOME/.cargo/bin" >> $GITHUB_PATH + source "$HOME/.cargo/env" + rustc --version + cargo --version + + - name: Build native library + shell: bash + run: | + ./paimon-diskann/paimon-diskann-jni/scripts/build-native.sh --clean + + - name: List built libraries (Linux AMD64) + if: inputs.platform == 'linux-amd64' + shell: bash + run: | + echo "=== Built libraries ===" + ls -la paimon-diskann/paimon-diskann-jni/src/main/resources/linux/amd64/ + echo "" + echo "=== Library dependencies ===" + ldd paimon-diskann/paimon-diskann-jni/src/main/resources/linux/amd64/libpaimon_diskann_jni.so || true + + - name: List built libraries (Linux AARCH64) + if: inputs.platform == 'linux-aarch64' + shell: bash + run: | + echo "=== Built libraries ===" + ls -la paimon-diskann/paimon-diskann-jni/src/main/resources/linux/aarch64/ + echo "" + echo "=== Library dependencies ===" + ldd paimon-diskann/paimon-diskann-jni/src/main/resources/linux/aarch64/libpaimon_diskann_jni.so || true + + - name: List built libraries (macOS) + if: inputs.platform == 'darwin-aarch64' + shell: bash + run: | + echo "=== Built libraries ===" + ls -la paimon-diskann/paimon-diskann-jni/src/main/resources/darwin/aarch64/ + echo "" + echo "=== Library dependencies ===" + otool -L paimon-diskann/paimon-diskann-jni/src/main/resources/darwin/aarch64/libpaimon_diskann_jni.dylib || true diff --git a/.github/workflows/build-diskann-native.yml b/.github/workflows/build-diskann-native.yml new file mode 100644 index 000000000000..0495fe50b7e6 --- /dev/null +++ b/.github/workflows/build-diskann-native.yml @@ -0,0 +1,68 @@ +################################################################################ +# 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: Build DiskANN Native Library + +on: + workflow_call: + inputs: + platform: + description: 'Target platform (linux-amd64, linux-aarch64, darwin-aarch64)' + required: false + type: string + default: 'linux-amd64' + jdk-version: + description: 'JDK version to use' + required: false + type: string + default: '8' + artifact-name: + description: 'Name for the uploaded artifact' + required: false + type: string + default: 'diskann-native-linux-amd64' + retention-days: + description: 'Number of days to retain the artifact' + required: false + type: number + default: 1 + +jobs: + build_native: + runs-on: ubuntu-latest + steps: + - name: Checkout code + uses: actions/checkout@v6 + + - name: Set up JDK ${{ inputs.jdk-version }} + uses: actions/setup-java@v5 + with: + java-version: ${{ inputs.jdk-version }} + distribution: 'temurin' + + - name: Build DiskANN native library + uses: ./.github/actions/build-diskann-native + with: + platform: ${{ inputs.platform }} + + - name: Upload native library artifact + uses: actions/upload-artifact@v6 + with: + name: ${{ inputs.artifact-name }} + path: paimon-diskann/paimon-diskann-jni/src/main/resources/linux/amd64/ + retention-days: ${{ inputs.retention-days }} diff --git a/.github/workflows/publish-diskann_snapshot.yml b/.github/workflows/publish-diskann_snapshot.yml new file mode 100644 index 000000000000..93384ea44eed --- /dev/null +++ b/.github/workflows/publish-diskann_snapshot.yml @@ -0,0 +1,184 @@ +################################################################################ +# 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: Publish DiskANN Snapshot + +on: + schedule: + # At the end of every day + - cron: '0 1 * * *' + workflow_dispatch: + push: + paths: + - 'paimon-diskann/**' + branches: + - master + +env: + JDK_VERSION: 8 + MAVEN_OPTS: -Dmaven.wagon.httpconnectionManager.ttlSeconds=30 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true + +concurrency: + group: ${{ github.workflow }}-${{ github.event_name }}-${{ github.event.number || github.run_id }} + cancel-in-progress: true + +jobs: + # Build native library for Linux AMD64 + build-native-linux-amd64: + runs-on: ubuntu-latest + steps: + - name: Checkout code + uses: actions/checkout@v6 + + - name: Set up JDK ${{ env.JDK_VERSION }} + uses: actions/setup-java@v5 + with: + java-version: ${{ env.JDK_VERSION }} + distribution: 'temurin' + + - name: Build DiskANN native library + uses: ./.github/actions/build-diskann-native + with: + platform: linux-amd64 + + - name: Upload native library + uses: actions/upload-artifact@v6 + with: + name: native-linux-amd64 + path: paimon-diskann/paimon-diskann-jni/src/main/resources/linux/amd64/ + retention-days: 1 + + # Build native library for Linux AARCH64 + build-native-linux-aarch64: + runs-on: ubuntu-24.04-arm + steps: + - name: Checkout code + uses: actions/checkout@v6 + + - name: Set up JDK ${{ env.JDK_VERSION }} + uses: actions/setup-java@v5 + with: + java-version: ${{ env.JDK_VERSION }} + distribution: 'temurin' + + - name: Build DiskANN native library + uses: ./.github/actions/build-diskann-native + with: + platform: linux-aarch64 + + - name: Upload native library + uses: actions/upload-artifact@v6 + with: + name: native-linux-aarch64 + path: paimon-diskann/paimon-diskann-jni/src/main/resources/linux/aarch64/ + retention-days: 1 + + # Build native library for macOS ARM (Apple Silicon) + build-native-macos-arm: + runs-on: macos-14 + steps: + - name: Checkout code + uses: actions/checkout@v6 + + - name: Set up JDK ${{ env.JDK_VERSION }} + uses: actions/setup-java@v5 + with: + java-version: ${{ env.JDK_VERSION }} + distribution: 'zulu' + + - name: Build DiskANN native library + uses: ./.github/actions/build-diskann-native + with: + platform: darwin-aarch64 + + - name: Upload native library + uses: actions/upload-artifact@v6 + with: + name: native-darwin-aarch64 + path: paimon-diskann/paimon-diskann-jni/src/main/resources/darwin/aarch64/ + retention-days: 1 + + # Package and publish + package-and-publish: + if: github.repository == 'apache/paimon' + needs: [build-native-linux-amd64, build-native-linux-aarch64, build-native-macos-arm] + runs-on: ubuntu-latest + steps: + - name: Checkout code + uses: actions/checkout@v6 + + - name: Set up JDK ${{ env.JDK_VERSION }} + uses: actions/setup-java@v5 + with: + java-version: ${{ env.JDK_VERSION }} + distribution: 'zulu' + + - name: Download Linux AMD64 native library + uses: actions/download-artifact@v7 + with: + name: native-linux-amd64 + path: paimon-diskann/paimon-diskann-jni/src/main/resources/linux/amd64/ + + - name: Download Linux AARCH64 native library + uses: actions/download-artifact@v7 + with: + name: native-linux-aarch64 + path: paimon-diskann/paimon-diskann-jni/src/main/resources/linux/aarch64/ + + - name: Download macOS ARM native library + uses: actions/download-artifact@v7 + with: + name: native-darwin-aarch64 + path: paimon-diskann/paimon-diskann-jni/src/main/resources/darwin/aarch64/ + + - name: List all native libraries + run: | + echo "=== All native libraries ===" + find paimon-diskann/paimon-diskann-jni/src/main/resources -type f \( -name "*.so" -o -name "*.so.*" -o -name "*.dylib" \) -exec ls -la {} \; + + - name: Cache local Maven repository + uses: actions/cache@v5 + with: + path: ~/.m2/repository + key: diskann-snapshot-maven-${{ hashFiles('**/pom.xml') }} + restore-keys: | + diskann-snapshot-maven- + + - name: Build and package paimon-diskann-jni + run: | + mvn -B -ntp clean install -pl paimon-diskann/paimon-diskann-jni -am -DskipTests -Ppaimon-diskann -Drat.skip + + - name: Build and package paimon-diskann-index + run: | + mvn -B -ntp clean install -pl paimon-diskann/paimon-diskann-index -am -DskipTests -Ppaimon-diskann -Drat.skip + + - name: Publish snapshot + env: + ASF_USERNAME: ${{ secrets.NEXUS_USER }} + ASF_PASSWORD: ${{ secrets.NEXUS_PW }} + MAVEN_OPTS: -Xmx4096m + run: | + tmp_settings="tmp-settings.xml" + echo "" > $tmp_settings + echo "apache.snapshots.https$ASF_USERNAME" >> $tmp_settings + echo "$ASF_PASSWORD" >> $tmp_settings + echo "" >> $tmp_settings + + mvn --settings $tmp_settings -ntp deploy -pl paimon-diskann/paimon-diskann-jni,paimon-diskann/paimon-diskann-index -Dgpg.skip -Drat.skip -DskipTests -Ppaimon-diskann + + rm $tmp_settings diff --git a/paimon-diskann/paimon-diskann-e2e-test/pom.xml b/paimon-diskann/paimon-diskann-e2e-test/pom.xml new file mode 100644 index 000000000000..d74a68a7de39 --- /dev/null +++ b/paimon-diskann/paimon-diskann-e2e-test/pom.xml @@ -0,0 +1,293 @@ + + + + 4.0.0 + + + org.apache.paimon + paimon-diskann + 1.4-SNAPSHOT + + + paimon-diskann-e2e-test + Paimon : DiskANN End to End Tests + + + java8 + 3.5 + 3.5.8 + + + + + + + + com.fasterxml.jackson.core + jackson-core + 2.15.2 + test + + + + com.fasterxml.jackson.core + jackson-databind + 2.15.2 + test + + + + com.fasterxml.jackson.core + jackson-annotations + 2.15.2 + test + + + + org.apache.paimon + paimon-format + ${project.version} + test + + + + org.apache.paimon + paimon-spark-${test.spark.main.version}_${scala.binary.version} + ${project.version} + test + + + + org.apache.paimon + paimon-diskann-index + ${project.version} + test + + + + org.apache.paimon + paimon-test-utils + ${project.version} + test + + + + org.apache.paimon + paimon-spark-ut_${scala.binary.version} + ${project.version} + test + test-jar + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${test.spark.version} + test + + + org.apache.logging.log4j + log4j-slf4j2-impl + + + org.slf4j + jul-to-slf4j + + + org.slf4j + jcl-over-slf4j + + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${test.spark.version} + tests + test + + + org.apache.logging.log4j + log4j-slf4j2-impl + + + org.slf4j + jul-to-slf4j + + + org.slf4j + jcl-over-slf4j + + + + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${test.spark.version} + tests + test + + + + org.apache.spark + spark-core_${scala.binary.version} + ${test.spark.version} + test + + + org.apache.logging.log4j + log4j-slf4j2-impl + + + org.slf4j + jul-to-slf4j + + + org.slf4j + jcl-over-slf4j + + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${test.spark.version} + tests + test + + + org.apache.logging.log4j + log4j-slf4j2-impl + + + org.slf4j + jul-to-slf4j + + + org.slf4j + jcl-over-slf4j + + + + + + org.apache.spark + spark-hive_${scala.binary.version} + ${test.spark.version} + test + + + + + org.scala-lang + scala-library + ${scala.version} + test + + + + + org.scalatest + scalatest_${scala.binary.version} + 3.2.14 + test + + + + + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + + net.alchim31.maven + scala-maven-plugin + ${scala-maven-plugin.version} + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + ${scala.version} + + -nobootcp + + + + + + org.scalatest + scalatest-maven-plugin + ${scalatest-maven-plugin.version} + + ${project.build.directory}/surefire-reports + . + TestSuiteReport.txt + -ea -Xmx4g -Xss4m -XX:MaxMetaspaceSize=2g -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true + org.apache.paimon.spark.sql.DiskAnnVectorIndexE2ETest + + + + test + + test + + + + + + + + + + java11 + + [11,) + + + java11 + + + + diff --git a/paimon-diskann/paimon-diskann-e2e-test/src/test/scala/org/apache/paimon/spark/sql/DiskAnnVectorIndexE2ETest.scala b/paimon-diskann/paimon-diskann-e2e-test/src/test/scala/org/apache/paimon/spark/sql/DiskAnnVectorIndexE2ETest.scala new file mode 100644 index 000000000000..ceb7279c4a35 --- /dev/null +++ b/paimon-diskann/paimon-diskann-e2e-test/src/test/scala/org/apache/paimon/spark/sql/DiskAnnVectorIndexE2ETest.scala @@ -0,0 +1,380 @@ +/* + * 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.paimon.spark.sql + +import org.apache.paimon.spark.PaimonSparkTestBase + +import scala.collection.JavaConverters._ + +/** End-to-end tests for DiskANN vector index read/write operations on Spark 3.5. */ +class DiskAnnVectorIndexE2ETest extends PaimonSparkTestBase { + + // ========== Index Creation Tests ========== + + test("create diskann vector index - basic") { + withTable("T") { + spark.sql(""" + |CREATE TABLE T (id INT, v ARRAY) + |TBLPROPERTIES ( + | 'bucket' = '-1', + | 'global-index.row-count-per-shard' = '10000', + | 'row-tracking.enabled' = 'true', + | 'data-evolution.enabled' = 'true') + |""".stripMargin) + + val values = (0 until 100) + .map( + i => s"($i, array(cast($i as float), cast(${i + 1} as float), cast(${i + 2} as float)))") + .mkString(",") + spark.sql(s"INSERT INTO T VALUES $values") + + val output = spark + .sql("CALL sys.create_global_index(table => 'test.T', index_column => 'v', index_type => 'diskann-vector-ann', options => 'vector.dim=3')") + .collect() + .head + assert(output.getBoolean(0)) + + val table = loadTable("T") + val indexEntries = table + .store() + .newIndexFileHandler() + .scanEntries() + .asScala + .filter(_.indexFile().indexType() == "diskann-vector-ann") + + assert(indexEntries.nonEmpty) + val totalRowCount = indexEntries.map(_.indexFile().rowCount()).sum + assert(totalRowCount == 100L) + } + } + + test("create diskann vector index - with different index types") { + withTable("T") { + spark.sql(""" + |CREATE TABLE T (id INT, v ARRAY) + |TBLPROPERTIES ( + | 'bucket' = '-1', + | 'global-index.row-count-per-shard' = '10000', + | 'row-tracking.enabled' = 'true', + | 'data-evolution.enabled' = 'true') + |""".stripMargin) + + val values = (0 until 50) + .map( + i => s"($i, array(cast($i as float), cast(${i + 1} as float), cast(${i + 2} as float)))") + .mkString(",") + spark.sql(s"INSERT INTO T VALUES $values") + + val output = spark + .sql("CALL sys.create_global_index(table => 'test.T', index_column => 'v', index_type => 'diskann-vector-ann', options => 'vector.dim=3,vector.diskann.index-type=MEMORY')") + .collect() + .head + assert(output.getBoolean(0)) + + val table = loadTable("T") + val indexEntries = table + .store() + .newIndexFileHandler() + .scanEntries() + .asScala + .filter(_.indexFile().indexType() == "diskann-vector-ann") + + assert(indexEntries.nonEmpty) + } + } + + test("create diskann vector index - with partitioned table") { + withTable("T") { + spark.sql(""" + |CREATE TABLE T (id INT, v ARRAY, pt STRING) + |TBLPROPERTIES ( + | 'bucket' = '-1', + | 'global-index.row-count-per-shard' = '10000', + | 'row-tracking.enabled' = 'true', + | 'data-evolution.enabled' = 'true') + | PARTITIONED BY (pt) + |""".stripMargin) + + var values = (0 until 500) + .map( + i => + s"($i, array(cast($i as float), cast(${i + 1} as float), cast(${i + 2} as float)), 'p0')") + .mkString(",") + spark.sql(s"INSERT INTO T VALUES $values") + + values = (0 until 300) + .map( + i => + s"($i, array(cast($i as float), cast(${i + 1} as float), cast(${i + 2} as float)), 'p1')") + .mkString(",") + spark.sql(s"INSERT INTO T VALUES $values") + + val output = spark + .sql("CALL sys.create_global_index(table => 'test.T', index_column => 'v', index_type => 'diskann-vector-ann', options => 'vector.dim=3')") + .collect() + .head + assert(output.getBoolean(0)) + + val table = loadTable("T") + val indexEntries = table + .store() + .newIndexFileHandler() + .scanEntries() + .asScala + .filter(_.indexFile().indexType() == "diskann-vector-ann") + + assert(indexEntries.nonEmpty) + val totalRowCount = indexEntries.map(_.indexFile().rowCount()).sum + assert(totalRowCount == 800L) + } + } + + // ========== Index Write Tests ========== + + test("write vectors - large dataset") { + withTable("T") { + spark.sql(""" + |CREATE TABLE T (id INT, v ARRAY) + |TBLPROPERTIES ( + | 'bucket' = '-1', + | 'global-index.row-count-per-shard' = '10000', + | 'row-tracking.enabled' = 'true', + | 'data-evolution.enabled' = 'true') + |""".stripMargin) + + val values = (0 until 10000) + .map( + i => s"($i, array(cast($i as float), cast(${i + 1} as float), cast(${i + 2} as float)))") + .mkString(",") + spark.sql(s"INSERT INTO T VALUES $values") + + val output = spark + .sql("CALL sys.create_global_index(table => 'test.T', index_column => 'v', index_type => 'diskann-vector-ann', options => 'vector.dim=3')") + .collect() + .head + assert(output.getBoolean(0)) + + val table = loadTable("T") + val indexEntries = table + .store() + .newIndexFileHandler() + .scanEntries() + .asScala + .filter(_.indexFile().indexType() == "diskann-vector-ann") + + val totalRowCount = indexEntries.map(_.indexFile().rowCount()).sum + assert(totalRowCount == 10000L) + } + } + + // ========== Index Read/Search Tests ========== + + test("read vectors - basic search") { + withTable("T") { + spark.sql(""" + |CREATE TABLE T (id INT, v ARRAY) + |TBLPROPERTIES ( + | 'bucket' = '-1', + | 'global-index.row-count-per-shard' = '10000', + | 'row-tracking.enabled' = 'true', + | 'data-evolution.enabled' = 'true') + |""".stripMargin) + + val values = (0 until 100) + .map( + i => s"($i, array(cast($i as float), cast(${i + 1} as float), cast(${i + 2} as float)))") + .mkString(",") + spark.sql(s"INSERT INTO T VALUES $values") + + spark + .sql("CALL sys.create_global_index(table => 'test.T', index_column => 'v', index_type => 'diskann-vector-ann', options => 'vector.dim=3')") + .collect() + + val result = spark + .sql(""" + |SELECT * FROM vector_search('T', 'v', array(50.0f, 51.0f, 52.0f), 5) + |""".stripMargin) + .collect() + assert(result.map(_.getInt(0)).contains(50)) + } + } + + test("read vectors - top-k search with different k values") { + withTable("T") { + spark.sql(""" + |CREATE TABLE T (id INT, v ARRAY) + |TBLPROPERTIES ( + | 'bucket' = '-1', + | 'global-index.row-count-per-shard' = '10000', + | 'row-tracking.enabled' = 'true', + | 'data-evolution.enabled' = 'true') + |""".stripMargin) + + val values = (0 until 200) + .map( + i => s"($i, array(cast($i as float), cast(${i + 1} as float), cast(${i + 2} as float)))") + .mkString(",") + spark.sql(s"INSERT INTO T VALUES $values") + + spark + .sql("CALL sys.create_global_index(table => 'test.T', index_column => 'v', index_type => 'diskann-vector-ann', options => 'vector.dim=3')") + .collect() + + var result = spark + .sql(""" + |SELECT * FROM vector_search('T', 'v', array(100.0f, 101.0f, 102.0f), 1) + |""".stripMargin) + .collect() + assert(result.length == 1) + assert(result.head.getInt(0) == 100) + + result = spark + .sql(""" + |SELECT * FROM vector_search('T', 'v', array(100.0f, 101.0f, 102.0f), 10) + |""".stripMargin) + .collect() + assert(result.map(_.getInt(0)).contains(100)) + } + } + + test("read vectors - multiple concurrent searches") { + withTable("T") { + spark.sql(""" + |CREATE TABLE T (id INT, v ARRAY) + |TBLPROPERTIES ( + | 'bucket' = '-1', + | 'global-index.row-count-per-shard' = '10000', + | 'row-tracking.enabled' = 'true', + | 'data-evolution.enabled' = 'true') + |""".stripMargin) + + val values = (0 until 500) + .map( + i => s"($i, array(cast($i as float), cast(${i + 1} as float), cast(${i + 2} as float)))") + .mkString(",") + spark.sql(s"INSERT INTO T VALUES $values") + + spark + .sql("CALL sys.create_global_index(table => 'test.T', index_column => 'v', index_type => 'diskann-vector-ann', options => 'vector.dim=3')") + .collect() + + val result1 = spark + .sql(""" + |SELECT * FROM vector_search('T', 'v', array(10.0f, 11.0f, 12.0f), 3) + |""".stripMargin) + .collect() + assert(result1.length == 3) + assert(result1.map(_.getInt(0)).contains(10)) + + val result2 = spark + .sql(""" + |SELECT * FROM vector_search('T', 'v', array(250.0f, 251.0f, 252.0f), 5) + |""".stripMargin) + .collect() + assert(result2.map(_.getInt(0)).contains(250)) + + val result3 = spark + .sql(""" + |SELECT * FROM vector_search('T', 'v', array(450.0f, 451.0f, 452.0f), 7) + |""".stripMargin) + .collect() + assert(result3.map(_.getInt(0)).contains(450)) + } + } + + test("read vectors - normalized vectors search") { + withTable("T") { + spark.sql(""" + |CREATE TABLE T (id INT, v ARRAY) + |TBLPROPERTIES ( + | 'bucket' = '-1', + | 'global-index.row-count-per-shard' = '10000', + | 'row-tracking.enabled' = 'true', + | 'data-evolution.enabled' = 'true') + |""".stripMargin) + + val values = (1 to 100) + .map { + i => + val v = math.sqrt(3.0 * i * i) + val normalized = i.toFloat / v.toFloat + s"($i, array($normalized, $normalized, $normalized))" + } + .mkString(",") + spark.sql(s"INSERT INTO T VALUES $values") + + spark.sql( + "CALL sys.create_global_index(table => 'test.T', index_column => 'v', index_type => 'diskann-vector-ann', options => 'vector.dim=3')") + + val result = spark + .sql(""" + |SELECT * FROM vector_search('T', 'v', array(0.577f, 0.577f, 0.577f), 10) + |""".stripMargin) + .collect() + + assert(result.length == 10) + } + } + + // ========== Integration Tests ========== + + test("end-to-end: write, index, read cycle") { + withTable("T") { + spark.sql(""" + |CREATE TABLE T (id INT, name STRING, embedding ARRAY) + |TBLPROPERTIES ( + | 'bucket' = '-1', + | 'global-index.row-count-per-shard' = '10000', + | 'row-tracking.enabled' = 'true', + | 'data-evolution.enabled' = 'true') + |""".stripMargin) + + val values = (0 until 1000) + .map( + i => + s"($i, 'item_$i', array(cast($i as float), cast(${i + 1} as float), cast(${i + 2} as float)))") + .mkString(",") + spark.sql(s"INSERT INTO T VALUES $values") + + val indexResult = spark + .sql("CALL sys.create_global_index(table => 'test.T', index_column => 'embedding', index_type => 'diskann-vector-ann', options => 'vector.dim=3')") + .collect() + .head + assert(indexResult.getBoolean(0)) + + val table = loadTable("T") + val indexEntries = table + .store() + .newIndexFileHandler() + .scanEntries() + .asScala + .filter(_.indexFile().indexType() == "diskann-vector-ann") + assert(indexEntries.nonEmpty) + + val searchResult = spark + .sql( + """ + |SELECT id, name FROM vector_search('T', 'embedding', array(500.0f, 501.0f, 502.0f), 10) + |""".stripMargin) + .collect() + + assert(searchResult.exists(row => row.getInt(0) == 500 && row.getString(1) == "item_500")) + } + } +} diff --git a/paimon-diskann/paimon-diskann-index/pom.xml b/paimon-diskann/paimon-diskann-index/pom.xml new file mode 100644 index 000000000000..c71637adcadb --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/pom.xml @@ -0,0 +1,108 @@ + + + + 4.0.0 + + + org.apache.paimon + paimon-diskann + 1.4-SNAPSHOT + + + paimon-diskann-index + Paimon : DiskANN Index + + + + org.apache.paimon + paimon-common + ${project.version} + + + + org.apache.paimon + paimon-diskann-jni + ${project.version} + + + + + org.junit.jupiter + junit-jupiter + ${junit5.version} + test + + + + org.apache.paimon + paimon-core + ${project.version} + test + + + + org.apache.paimon + paimon-format + ${project.version} + test + + + + org.apache.paimon + paimon-test-utils + ${project.version} + test + + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + test + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + + + + + + maven-jar-plugin + + + + test-jar + + + + + + + diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java new file mode 100644 index 000000000000..a83c23ed5e3d --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java @@ -0,0 +1,189 @@ +/* + * 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.paimon.diskann.index; + +import org.apache.paimon.diskann.Index; +import org.apache.paimon.diskann.MetricType; + +import java.io.Closeable; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * A wrapper class for DiskANN index with zero-copy support. + * + *

This class provides a safe Java API for interacting with native DiskANN indices using direct + * ByteBuffers for zero-copy data transfer. + */ +public class DiskAnnIndex implements Closeable { + + private final Index index; + private final int dimension; + private final DiskAnnVectorMetric metric; + private final DiskAnnIndexType indexType; + private volatile boolean closed = false; + + private DiskAnnIndex( + Index index, int dimension, DiskAnnVectorMetric metric, DiskAnnIndexType indexType) { + this.index = index; + this.dimension = dimension; + this.metric = metric; + this.indexType = indexType; + } + + public static DiskAnnIndex create( + int dimension, + DiskAnnVectorMetric metric, + DiskAnnIndexType indexType, + int maxDegree, + int buildListSize) { + MetricType metricType = metric.toMetricType(); + Index index = + Index.create(dimension, metricType, indexType.value(), maxDegree, buildListSize); + return new DiskAnnIndex(index, dimension, metric, indexType); + } + + public void addWithIds(ByteBuffer vectorBuffer, ByteBuffer idBuffer, int n) { + ensureOpen(); + validateVectorBuffer(vectorBuffer, n); + validateIdBuffer(idBuffer, n); + index.addWithIds(n, vectorBuffer, idBuffer); + } + + public void build(int buildListSize) { + ensureOpen(); + index.build(buildListSize); + } + + public void search( + float[] queryVectors, + int n, + int k, + int searchListSize, + float[] distances, + long[] labels) { + ensureOpen(); + if (queryVectors.length < n * dimension) { + throw new IllegalArgumentException( + "Query vectors array too small: required " + + (n * dimension) + + ", got " + + queryVectors.length); + } + if (distances.length < n * k) { + throw new IllegalArgumentException( + "Distances array too small: required " + (n * k) + ", got " + distances.length); + } + if (labels.length < n * k) { + throw new IllegalArgumentException( + "Labels array too small: required " + (n * k) + ", got " + labels.length); + } + index.search(n, queryVectors, k, searchListSize, distances, labels); + } + + public long size() { + ensureOpen(); + return index.getCount(); + } + + public int dimension() { + return dimension; + } + + public DiskAnnVectorMetric metric() { + return metric; + } + + public DiskAnnIndexType indexType() { + return indexType; + } + + public long serializeSize() { + ensureOpen(); + return index.serializeSize(); + } + + public long serialize(ByteBuffer buffer) { + ensureOpen(); + if (!buffer.isDirect()) { + throw new IllegalArgumentException("Buffer must be a direct buffer"); + } + return index.serialize(buffer); + } + + public static DiskAnnIndex deserialize(byte[] data, DiskAnnVectorMetric metric) { + Index index = Index.deserialize(data); + return new DiskAnnIndex(index, index.getDimension(), metric, DiskAnnIndexType.UNKNOWN); + } + + public static ByteBuffer allocateVectorBuffer(int numVectors, int dimension) { + return ByteBuffer.allocateDirect(numVectors * dimension * Float.BYTES) + .order(ByteOrder.nativeOrder()); + } + + public static ByteBuffer allocateIdBuffer(int numIds) { + return ByteBuffer.allocateDirect(numIds * Long.BYTES).order(ByteOrder.nativeOrder()); + } + + private void validateVectorBuffer(ByteBuffer buffer, int numVectors) { + if (!buffer.isDirect()) { + throw new IllegalArgumentException("Vector buffer must be a direct buffer"); + } + int requiredBytes = numVectors * dimension * Float.BYTES; + if (buffer.capacity() < requiredBytes) { + throw new IllegalArgumentException( + "Vector buffer too small: required " + + requiredBytes + + " bytes, got " + + buffer.capacity()); + } + } + + private void validateIdBuffer(ByteBuffer buffer, int numIds) { + if (!buffer.isDirect()) { + throw new IllegalArgumentException("ID buffer must be a direct buffer"); + } + int requiredBytes = numIds * Long.BYTES; + if (buffer.capacity() < requiredBytes) { + throw new IllegalArgumentException( + "ID buffer too small: required " + + requiredBytes + + " bytes, got " + + buffer.capacity()); + } + } + + private void ensureOpen() { + if (closed) { + throw new IllegalStateException("Index has been closed"); + } + } + + @Override + public void close() { + if (!closed) { + synchronized (this) { + if (!closed) { + index.close(); + closed = true; + } + } + } + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexMeta.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexMeta.java new file mode 100644 index 000000000000..aa3af7738a65 --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexMeta.java @@ -0,0 +1,106 @@ +/* + * 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.paimon.diskann.index; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.Serializable; + +/** Metadata for DiskANN vector index. */ +public class DiskAnnIndexMeta implements Serializable { + + private static final long serialVersionUID = 1L; + + private static final int VERSION = 1; + + private final int dim; + private final int metricValue; + private final int indexTypeValue; + private final long numVectors; + private final long minId; + private final long maxId; + + public DiskAnnIndexMeta( + int dim, int metricValue, int indexTypeValue, long numVectors, long minId, long maxId) { + this.dim = dim; + this.metricValue = metricValue; + this.indexTypeValue = indexTypeValue; + this.numVectors = numVectors; + this.minId = minId; + this.maxId = maxId; + } + + public int dim() { + return dim; + } + + public int metricValue() { + return metricValue; + } + + public int indexTypeValue() { + return indexTypeValue; + } + + public long numVectors() { + return numVectors; + } + + public long minId() { + return minId; + } + + public long maxId() { + return maxId; + } + + /** Serialize metadata to byte array. */ + public byte[] serialize() throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos); + out.writeInt(VERSION); + out.writeInt(dim); + out.writeInt(metricValue); + out.writeInt(indexTypeValue); + out.writeLong(numVectors); + out.writeLong(minId); + out.writeLong(maxId); + out.flush(); + return baos.toByteArray(); + } + + /** Deserialize metadata from byte array. */ + public static DiskAnnIndexMeta deserialize(byte[] data) throws IOException { + DataInputStream in = new DataInputStream(new ByteArrayInputStream(data)); + int version = in.readInt(); + if (version != VERSION) { + throw new IOException("Unsupported DiskANN index meta version: " + version); + } + int dim = in.readInt(); + int metricValue = in.readInt(); + int indexTypeValue = in.readInt(); + long numVectors = in.readLong(); + long minId = in.readLong(); + long maxId = in.readLong(); + return new DiskAnnIndexMeta(dim, metricValue, indexTypeValue, numVectors, minId, maxId); + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexType.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexType.java new file mode 100644 index 000000000000..101330164777 --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexType.java @@ -0,0 +1,36 @@ +/* + * 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.paimon.diskann.index; + +/** DiskANN index type. */ +public enum DiskAnnIndexType { + MEMORY(0), + DISK(1), + UNKNOWN(-1); + + private final int value; + + DiskAnnIndexType(int value) { + this.value = value; + } + + public int value() { + return value; + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnScoredGlobalIndexResult.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnScoredGlobalIndexResult.java new file mode 100644 index 000000000000..c5bf687c02d1 --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnScoredGlobalIndexResult.java @@ -0,0 +1,48 @@ +/* + * 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.paimon.diskann.index; + +import org.apache.paimon.globalindex.ScoreGetter; +import org.apache.paimon.globalindex.ScoredGlobalIndexResult; +import org.apache.paimon.utils.RoaringNavigableMap64; + +import java.util.HashMap; + +/** Vector search global index result for DiskANN vector index. */ +public class DiskAnnScoredGlobalIndexResult implements ScoredGlobalIndexResult { + + private final HashMap id2scores; + private final RoaringNavigableMap64 results; + + public DiskAnnScoredGlobalIndexResult( + RoaringNavigableMap64 results, HashMap id2scores) { + this.id2scores = id2scores; + this.results = results; + } + + @Override + public ScoreGetter scoreGetter() { + return id2scores::get; + } + + @Override + public RoaringNavigableMap64 results() { + return this.results; + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java new file mode 100644 index 000000000000..1c505fa91509 --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java @@ -0,0 +1,392 @@ +/* + * 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.paimon.diskann.index; + +import org.apache.paimon.fs.SeekableInputStream; +import org.apache.paimon.globalindex.GlobalIndexIOMeta; +import org.apache.paimon.globalindex.GlobalIndexReader; +import org.apache.paimon.globalindex.GlobalIndexResult; +import org.apache.paimon.globalindex.io.GlobalIndexFileReader; +import org.apache.paimon.predicate.FieldRef; +import org.apache.paimon.predicate.VectorSearch; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.FloatType; +import org.apache.paimon.utils.IOUtils; +import org.apache.paimon.utils.RoaringNavigableMap64; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Optional; +import java.util.PriorityQueue; + +/** + * Vector global index reader using DiskANN. + * + *

This implementation uses DiskANN for efficient approximate nearest neighbor search. + */ +public class DiskAnnVectorGlobalIndexReader implements GlobalIndexReader { + + private final List indices; + private final List indexMetas; + private final List ioMetas; + private final GlobalIndexFileReader fileReader; + private final DataType fieldType; + private final DiskAnnVectorIndexOptions options; + private volatile boolean metasLoaded = false; + private volatile boolean indicesLoaded = false; + + public DiskAnnVectorGlobalIndexReader( + GlobalIndexFileReader fileReader, + List ioMetas, + DataType fieldType, + DiskAnnVectorIndexOptions options) { + this.fileReader = fileReader; + this.ioMetas = ioMetas; + this.fieldType = fieldType; + this.options = options; + this.indices = new ArrayList<>(); + this.indexMetas = new ArrayList<>(); + } + + @Override + public Optional visitVectorSearch(VectorSearch vectorSearch) { + try { + ensureLoadMetas(); + + RoaringNavigableMap64 includeRowIds = vectorSearch.includeRowIds(); + if (includeRowIds != null) { + List matchingIndices = new ArrayList<>(); + for (int i = 0; i < indexMetas.size(); i++) { + DiskAnnIndexMeta meta = indexMetas.get(i); + if (hasOverlap(meta.minId(), meta.maxId(), includeRowIds)) { + matchingIndices.add(i); + } + } + if (matchingIndices.isEmpty()) { + return Optional.empty(); + } + ensureLoadIndices(matchingIndices); + } else { + ensureLoadAllIndices(); + } + + return Optional.ofNullable(search(vectorSearch)); + } catch (IOException e) { + throw new RuntimeException( + String.format( + "Failed to search DiskANN vector index with fieldName=%s, limit=%d", + vectorSearch.fieldName(), vectorSearch.limit()), + e); + } + } + + private boolean hasOverlap(long minId, long maxId, RoaringNavigableMap64 includeRowIds) { + for (Long id : includeRowIds) { + if (id >= minId && id <= maxId) { + return true; + } + if (id > maxId) { + break; + } + } + return false; + } + + private GlobalIndexResult search(VectorSearch vectorSearch) throws IOException { + validateVectorType(vectorSearch.vector()); + float[] queryVector = ((float[]) vectorSearch.vector()).clone(); + if (options.normalize()) { + normalizeL2(queryVector); + } + int limit = vectorSearch.limit(); + + PriorityQueue result = + new PriorityQueue<>(Comparator.comparingDouble(sr -> sr.score)); + + RoaringNavigableMap64 includeRowIds = vectorSearch.includeRowIds(); + int searchK = limit; + if (includeRowIds != null) { + searchK = + Math.max( + limit * options.searchFactor(), + (int) includeRowIds.getLongCardinality()); + } + + for (DiskAnnIndex index : indices) { + if (index == null) { + continue; + } + int effectiveK = (int) Math.min(searchK, index.size()); + if (effectiveK <= 0) { + continue; + } + + float[] distances = new float[effectiveK]; + long[] labels = new long[effectiveK]; + + index.search(queryVector, 1, effectiveK, options.searchListSize(), distances, labels); + + for (int i = 0; i < effectiveK; i++) { + long rowId = labels[i]; + if (rowId < 0) { + continue; + } + if (includeRowIds != null && !includeRowIds.contains(rowId)) { + continue; + } + float score = convertDistanceToScore(distances[i]); + + if (result.size() < limit) { + result.offer(new ScoredRow(rowId, score)); + } else { + if (result.peek() != null && score > result.peek().score) { + result.poll(); + result.offer(new ScoredRow(rowId, score)); + } + } + } + } + + RoaringNavigableMap64 roaringBitmap64 = new RoaringNavigableMap64(); + HashMap id2scores = new HashMap<>(result.size()); + for (ScoredRow scoredRow : result) { + id2scores.put(scoredRow.rowId, scoredRow.score); + roaringBitmap64.add(scoredRow.rowId); + } + return new DiskAnnScoredGlobalIndexResult(roaringBitmap64, id2scores); + } + + private float convertDistanceToScore(float distance) { + if (options.metric() == DiskAnnVectorMetric.L2 + || options.metric() == DiskAnnVectorMetric.COSINE) { + return 1.0f / (1.0f + distance); + } else { + return distance; + } + } + + private void validateVectorType(Object vector) { + if (!(vector instanceof float[])) { + throw new IllegalArgumentException( + "Expected float[] vector but got: " + vector.getClass()); + } + if (!(fieldType instanceof ArrayType) + || !(((ArrayType) fieldType).getElementType() instanceof FloatType)) { + throw new IllegalArgumentException( + "DiskANN currently only supports float arrays, but field type is: " + + fieldType); + } + } + + private void ensureLoadMetas() throws IOException { + if (!metasLoaded) { + synchronized (this) { + if (!metasLoaded) { + for (GlobalIndexIOMeta ioMeta : ioMetas) { + byte[] metaBytes = ioMeta.metadata(); + DiskAnnIndexMeta meta = DiskAnnIndexMeta.deserialize(metaBytes); + indexMetas.add(meta); + } + metasLoaded = true; + } + } + } + } + + private void ensureLoadAllIndices() throws IOException { + if (!indicesLoaded) { + synchronized (this) { + if (!indicesLoaded) { + for (int i = 0; i < ioMetas.size(); i++) { + loadIndexAt(i); + } + indicesLoaded = true; + } + } + } + } + + private void ensureLoadIndices(List positions) throws IOException { + synchronized (this) { + while (indices.size() < ioMetas.size()) { + indices.add(null); + } + for (int pos : positions) { + if (indices.get(pos) == null) { + loadIndexAt(pos); + } + } + } + } + + private void loadIndexAt(int position) throws IOException { + GlobalIndexIOMeta ioMeta = ioMetas.get(position); + DiskAnnIndex index = null; + try (SeekableInputStream in = fileReader.getInputStream(ioMeta)) { + index = loadIndex(in); + if (indices.size() <= position) { + while (indices.size() < position) { + indices.add(null); + } + indices.add(index); + } else { + indices.set(position, index); + } + } catch (Exception e) { + IOUtils.closeQuietly(index); + throw e; + } + } + + private DiskAnnIndex loadIndex(SeekableInputStream in) throws IOException { + byte[] data = IOUtils.readFully(in, true); + return DiskAnnIndex.deserialize(data, options.metric()); + } + + private void normalizeL2(float[] vector) { + float norm = 0.0f; + for (float v : vector) { + norm += v * v; + } + norm = (float) Math.sqrt(norm); + if (norm > 0) { + for (int i = 0; i < vector.length; i++) { + vector[i] /= norm; + } + } + } + + @Override + public void close() throws IOException { + Throwable firstException = null; + + for (DiskAnnIndex index : indices) { + if (index == null) { + continue; + } + try { + index.close(); + } catch (Throwable t) { + if (firstException == null) { + firstException = t; + } else { + firstException.addSuppressed(t); + } + } + } + indices.clear(); + + if (firstException != null) { + if (firstException instanceof IOException) { + throw (IOException) firstException; + } else if (firstException instanceof RuntimeException) { + throw (RuntimeException) firstException; + } else { + throw new RuntimeException( + "Failed to close DiskANN vector global index reader", firstException); + } + } + } + + private static class ScoredRow { + final long rowId; + final float score; + + ScoredRow(long rowId, float score) { + this.rowId = rowId; + this.score = score; + } + } + + // =================== unsupported ===================== + + @Override + public Optional visitIsNotNull(FieldRef fieldRef) { + return Optional.empty(); + } + + @Override + public Optional visitIsNull(FieldRef fieldRef) { + return Optional.empty(); + } + + @Override + public Optional visitStartsWith(FieldRef fieldRef, Object literal) { + return Optional.empty(); + } + + @Override + public Optional visitEndsWith(FieldRef fieldRef, Object literal) { + return Optional.empty(); + } + + @Override + public Optional visitContains(FieldRef fieldRef, Object literal) { + return Optional.empty(); + } + + @Override + public Optional visitLike(FieldRef fieldRef, Object literal) { + return Optional.empty(); + } + + @Override + public Optional visitLessThan(FieldRef fieldRef, Object literal) { + return Optional.empty(); + } + + @Override + public Optional visitGreaterOrEqual(FieldRef fieldRef, Object literal) { + return Optional.empty(); + } + + @Override + public Optional visitNotEqual(FieldRef fieldRef, Object literal) { + return Optional.empty(); + } + + @Override + public Optional visitLessOrEqual(FieldRef fieldRef, Object literal) { + return Optional.empty(); + } + + @Override + public Optional visitEqual(FieldRef fieldRef, Object literal) { + return Optional.empty(); + } + + @Override + public Optional visitGreaterThan(FieldRef fieldRef, Object literal) { + return Optional.empty(); + } + + @Override + public Optional visitIn(FieldRef fieldRef, List literals) { + return Optional.empty(); + } + + @Override + public Optional visitNotIn(FieldRef fieldRef, List literals) { + return Optional.empty(); + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java new file mode 100644 index 000000000000..16d709345d32 --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java @@ -0,0 +1,260 @@ +/* + * 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.paimon.diskann.index; + +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.globalindex.GlobalIndexSingletonWriter; +import org.apache.paimon.globalindex.ResultEntry; +import org.apache.paimon.globalindex.io.GlobalIndexFileWriter; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.FloatType; + +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.FloatBuffer; +import java.nio.LongBuffer; +import java.util.ArrayList; +import java.util.List; + +/** + * Vector global index writer using DiskANN. + * + *

Vectors are added to the index in batches. When the current index reaches {@code sizePerIndex} + * vectors, it is built and serialized to a file and a new index is created. + */ +public class DiskAnnVectorGlobalIndexWriter implements GlobalIndexSingletonWriter { + + private static final int DEFAULT_BATCH_SIZE = 10000; + + private final GlobalIndexFileWriter fileWriter; + private final DiskAnnVectorIndexOptions options; + private final int sizePerIndex; + private final int batchSize; + private final int dim; + private final DataType fieldType; + + private long count = 0; + private long currentIndexCount = 0; + private long currentIndexMinId = Long.MAX_VALUE; + private long currentIndexMaxId = Long.MIN_VALUE; + private final List pendingBatch; + private final List results; + private DiskAnnIndex currentIndex; + private boolean built = false; + + public DiskAnnVectorGlobalIndexWriter( + GlobalIndexFileWriter fileWriter, + DataType fieldType, + DiskAnnVectorIndexOptions options) { + this.fileWriter = fileWriter; + this.fieldType = fieldType; + this.options = options; + this.sizePerIndex = options.sizePerIndex(); + this.batchSize = Math.min(DEFAULT_BATCH_SIZE, sizePerIndex); + this.dim = options.dimension(); + this.pendingBatch = new ArrayList<>(batchSize); + this.results = new ArrayList<>(); + + validateFieldType(fieldType); + } + + private void validateFieldType(DataType dataType) { + if (!(dataType instanceof ArrayType)) { + throw new IllegalArgumentException( + "DiskANN vector index requires ArrayType, but got: " + dataType); + } + DataType elementType = ((ArrayType) dataType).getElementType(); + if (!(elementType instanceof FloatType)) { + throw new IllegalArgumentException( + "DiskANN vector index requires float array, but got: " + elementType); + } + } + + @Override + public void write(Object fieldData) { + float[] vector; + if (fieldData instanceof float[]) { + vector = (float[]) fieldData; + } else if (fieldData instanceof InternalArray) { + vector = ((InternalArray) fieldData).toFloatArray(); + } else { + throw new RuntimeException( + "Unsupported vector type: " + fieldData.getClass().getName()); + } + checkDimension(vector); + if (options.normalize()) { + normalizeL2(vector); + } + currentIndexMinId = Math.min(currentIndexMinId, count); + currentIndexMaxId = Math.max(currentIndexMaxId, count); + pendingBatch.add(new VectorEntry(count, vector)); + count++; + + try { + if (pendingBatch.size() >= batchSize) { + addBatchToIndex(); + } + if (currentIndexCount >= sizePerIndex) { + flushCurrentIndex(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public List finish() { + try { + if (!pendingBatch.isEmpty()) { + addBatchToIndex(); + } + if (currentIndex != null && currentIndexCount > 0) { + flushCurrentIndex(); + } + return results; + } catch (IOException e) { + throw new RuntimeException("Failed to write DiskANN vector global index", e); + } + } + + private void addBatchToIndex() throws IOException { + if (pendingBatch.isEmpty()) { + return; + } + + if (currentIndex == null) { + currentIndex = createIndex(); + built = false; + } + + int n = pendingBatch.size(); + ByteBuffer vectorBuffer = DiskAnnIndex.allocateVectorBuffer(n, dim); + ByteBuffer idBuffer = DiskAnnIndex.allocateIdBuffer(n); + FloatBuffer floatView = vectorBuffer.asFloatBuffer(); + LongBuffer longView = idBuffer.asLongBuffer(); + + for (int i = 0; i < n; i++) { + VectorEntry entry = pendingBatch.get(i); + float[] vector = entry.vector; + for (int j = 0; j < dim; j++) { + floatView.put(i * dim + j, vector[j]); + } + longView.put(i, entry.id); + } + + currentIndex.addWithIds(vectorBuffer, idBuffer, n); + currentIndexCount += n; + pendingBatch.clear(); + built = false; + } + + private void flushCurrentIndex() throws IOException { + if (currentIndex == null || currentIndexCount == 0) { + return; + } + + if (!built) { + currentIndex.build(options.buildListSize()); + built = true; + } + + String fileName = fileWriter.newFileName(DiskAnnVectorGlobalIndexerFactory.IDENTIFIER); + try (OutputStream out = new BufferedOutputStream(fileWriter.newOutputStream(fileName))) { + long serializeSize = currentIndex.serializeSize(); + if (serializeSize > Integer.MAX_VALUE) { + throw new IOException( + "Index too large to serialize: " + + serializeSize + + " bytes exceeds maximum buffer size"); + } + ByteBuffer serializeBuffer = + ByteBuffer.allocateDirect((int) serializeSize).order(ByteOrder.nativeOrder()); + long bytesWritten = currentIndex.serialize(serializeBuffer); + + byte[] indexData = new byte[(int) bytesWritten]; + serializeBuffer.rewind(); + serializeBuffer.get(indexData); + out.write(indexData); + out.flush(); + } + + DiskAnnIndexMeta meta = + new DiskAnnIndexMeta( + dim, + options.metric().toMetricType().value(), + options.indexType().value(), + currentIndexCount, + currentIndexMinId, + currentIndexMaxId); + results.add(new ResultEntry(fileName, currentIndexCount, meta.serialize())); + + currentIndex.close(); + currentIndex = null; + currentIndexCount = 0; + currentIndexMinId = Long.MAX_VALUE; + currentIndexMaxId = Long.MIN_VALUE; + built = false; + } + + private DiskAnnIndex createIndex() { + return DiskAnnIndex.create( + options.dimension(), + options.metric(), + options.indexType(), + options.maxDegree(), + options.buildListSize()); + } + + private void checkDimension(float[] vector) { + if (vector.length != options.dimension()) { + throw new IllegalArgumentException( + String.format( + "Vector dimension mismatch: expected %d, but got %d", + options.dimension(), vector.length)); + } + } + + private void normalizeL2(float[] vector) { + float norm = 0.0f; + for (float v : vector) { + norm += v * v; + } + norm = (float) Math.sqrt(norm); + if (norm > 0) { + for (int i = 0; i < vector.length; i++) { + vector[i] /= norm; + } + } + } + + /** Entry holding a vector and its row ID. */ + private static class VectorEntry { + final long id; + final float[] vector; + + VectorEntry(long id, float[] vector) { + this.id = id; + this.vector = vector; + } + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexer.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexer.java new file mode 100644 index 000000000000..4efc00d3a088 --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexer.java @@ -0,0 +1,53 @@ +/* + * 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.paimon.diskann.index; + +import org.apache.paimon.globalindex.GlobalIndexIOMeta; +import org.apache.paimon.globalindex.GlobalIndexReader; +import org.apache.paimon.globalindex.GlobalIndexWriter; +import org.apache.paimon.globalindex.GlobalIndexer; +import org.apache.paimon.globalindex.io.GlobalIndexFileReader; +import org.apache.paimon.globalindex.io.GlobalIndexFileWriter; +import org.apache.paimon.options.Options; +import org.apache.paimon.types.DataType; + +import java.util.List; + +/** DiskANN vector global indexer. */ +public class DiskAnnVectorGlobalIndexer implements GlobalIndexer { + + private final DataType fieldType; + private final DiskAnnVectorIndexOptions options; + + public DiskAnnVectorGlobalIndexer(DataType fieldType, Options options) { + this.fieldType = fieldType; + this.options = new DiskAnnVectorIndexOptions(options); + } + + @Override + public GlobalIndexWriter createWriter(GlobalIndexFileWriter fileWriter) { + return new DiskAnnVectorGlobalIndexWriter(fileWriter, fieldType, options); + } + + @Override + public GlobalIndexReader createReader( + GlobalIndexFileReader fileReader, List files) { + return new DiskAnnVectorGlobalIndexReader(fileReader, files, fieldType, options); + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexerFactory.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexerFactory.java new file mode 100644 index 000000000000..df3d7274744c --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexerFactory.java @@ -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.paimon.diskann.index; + +import org.apache.paimon.globalindex.GlobalIndexer; +import org.apache.paimon.globalindex.GlobalIndexerFactory; +import org.apache.paimon.options.Options; +import org.apache.paimon.types.DataField; + +/** Factory for creating DiskANN vector index. */ +public class DiskAnnVectorGlobalIndexerFactory implements GlobalIndexerFactory { + + public static final String IDENTIFIER = "diskann-vector-ann"; + + @Override + public String identifier() { + return IDENTIFIER; + } + + @Override + public GlobalIndexer create(DataField field, Options options) { + return new DiskAnnVectorGlobalIndexer(field.type(), options); + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorIndexOptions.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorIndexOptions.java new file mode 100644 index 000000000000..e7da850cd1de --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorIndexOptions.java @@ -0,0 +1,147 @@ +/* + * 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.paimon.diskann.index; + +import org.apache.paimon.options.ConfigOption; +import org.apache.paimon.options.ConfigOptions; +import org.apache.paimon.options.Options; + +/** Options for DiskANN vector index. */ +public class DiskAnnVectorIndexOptions { + + public static final ConfigOption VECTOR_DIM = + ConfigOptions.key("vector.dim") + .intType() + .defaultValue(128) + .withDescription("The dimension of the vector"); + + public static final ConfigOption VECTOR_METRIC = + ConfigOptions.key("vector.metric") + .enumType(DiskAnnVectorMetric.class) + .defaultValue(DiskAnnVectorMetric.L2) + .withDescription( + "The similarity metric for vector search (L2, INNER_PRODUCT, COSINE), and L2 is the default"); + + public static final ConfigOption VECTOR_INDEX_TYPE = + ConfigOptions.key("vector.diskann.index-type") + .enumType(DiskAnnIndexType.class) + .defaultValue(DiskAnnIndexType.MEMORY) + .withDescription("The DiskANN index type to use (MEMORY, DISK)"); + + public static final ConfigOption VECTOR_MAX_DEGREE = + ConfigOptions.key("vector.diskann.max-degree") + .intType() + .defaultValue(64) + .withDescription("The maximum degree (R) for DiskANN graph construction"); + + public static final ConfigOption VECTOR_BUILD_LIST_SIZE = + ConfigOptions.key("vector.diskann.build-list-size") + .intType() + .defaultValue(100) + .withDescription("The build list size (L) for DiskANN index construction"); + + public static final ConfigOption VECTOR_SEARCH_LIST_SIZE = + ConfigOptions.key("vector.diskann.search-list-size") + .intType() + .defaultValue(100) + .withDescription("The search list size (L) for DiskANN query"); + + public static final ConfigOption VECTOR_SIZE_PER_INDEX = + ConfigOptions.key("vector.size-per-index") + .intType() + .defaultValue(200_0000) + .withDescription("The size of vectors stored in each vector index file"); + + public static final ConfigOption VECTOR_SEARCH_FACTOR = + ConfigOptions.key("vector.search-factor") + .intType() + .defaultValue(10) + .withDescription( + "The multiplier for the search limit when filtering is applied. " + + "This is used to fetch more results to ensure enough records after filtering."); + + public static final ConfigOption VECTOR_NORMALIZE = + ConfigOptions.key("vector.normalize") + .booleanType() + .defaultValue(false) + .withDescription( + "Whether to L2 normalize vectors before indexing and searching. " + + "Enabling this converts L2 distance to cosine similarity."); + + private final int dimension; + private final DiskAnnVectorMetric metric; + private final DiskAnnIndexType indexType; + private final int maxDegree; + private final int buildListSize; + private final int searchListSize; + private final int sizePerIndex; + private final int searchFactor; + private final boolean normalize; + + public DiskAnnVectorIndexOptions(Options options) { + this.dimension = options.get(VECTOR_DIM); + this.metric = options.get(VECTOR_METRIC); + this.indexType = options.get(VECTOR_INDEX_TYPE); + this.maxDegree = options.get(VECTOR_MAX_DEGREE); + this.buildListSize = options.get(VECTOR_BUILD_LIST_SIZE); + this.searchListSize = options.get(VECTOR_SEARCH_LIST_SIZE); + this.sizePerIndex = + options.get(VECTOR_SIZE_PER_INDEX) > 0 + ? options.get(VECTOR_SIZE_PER_INDEX) + : VECTOR_SIZE_PER_INDEX.defaultValue(); + this.searchFactor = options.get(VECTOR_SEARCH_FACTOR); + this.normalize = options.get(VECTOR_NORMALIZE); + } + + public int dimension() { + return dimension; + } + + public DiskAnnVectorMetric metric() { + return metric; + } + + public DiskAnnIndexType indexType() { + return indexType; + } + + public int maxDegree() { + return maxDegree; + } + + public int buildListSize() { + return buildListSize; + } + + public int searchListSize() { + return searchListSize; + } + + public int sizePerIndex() { + return sizePerIndex; + } + + public int searchFactor() { + return searchFactor; + } + + public boolean normalize() { + return normalize; + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorMetric.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorMetric.java new file mode 100644 index 000000000000..6336c3b7d5e9 --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorMetric.java @@ -0,0 +1,38 @@ +/* + * 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.paimon.diskann.index; + +import org.apache.paimon.diskann.MetricType; + +/** Metric type for DiskANN vector index. */ +public enum DiskAnnVectorMetric { + L2(MetricType.L2), + INNER_PRODUCT(MetricType.INNER_PRODUCT), + COSINE(MetricType.COSINE); + + private final MetricType metricType; + + DiskAnnVectorMetric(MetricType metricType) { + this.metricType = metricType; + } + + MetricType toMetricType() { + return metricType; + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/main/resources/META-INF/services/org.apache.paimon.globalindex.GlobalIndexerFactory b/paimon-diskann/paimon-diskann-index/src/main/resources/META-INF/services/org.apache.paimon.globalindex.GlobalIndexerFactory new file mode 100644 index 000000000000..9906dcfa8e46 --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/resources/META-INF/services/org.apache.paimon.globalindex.GlobalIndexerFactory @@ -0,0 +1,17 @@ +# 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. + +org.apache.paimon.diskann.index.DiskAnnVectorGlobalIndexerFactory diff --git a/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexScanTest.java b/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexScanTest.java new file mode 100644 index 000000000000..d2e2df289127 --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexScanTest.java @@ -0,0 +1,334 @@ +/* + * 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.paimon.diskann.index; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.GenericArray; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.diskann.DiskAnn; +import org.apache.paimon.diskann.DiskAnnException; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.PositionOutputStream; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.globalindex.ResultEntry; +import org.apache.paimon.globalindex.io.GlobalIndexFileWriter; +import org.apache.paimon.index.GlobalIndexMeta; +import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.io.CompactIncrement; +import org.apache.paimon.io.DataIncrement; +import org.apache.paimon.options.Options; +import org.apache.paimon.predicate.VectorSearch; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.table.sink.CommitMessageImpl; +import org.apache.paimon.table.sink.StreamTableCommit; +import org.apache.paimon.table.sink.StreamTableWrite; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.table.source.TableScan; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.junit.jupiter.api.Assumptions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.UUID; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for scanning DiskANN vector global index. */ +public class DiskAnnVectorGlobalIndexScanTest { + + @TempDir java.nio.file.Path tempDir; + + private FileStoreTable table; + private String commitUser; + private FileIO fileIO; + private RowType rowType; + private final String vectorFieldName = "vec"; + + @BeforeEach + public void before() throws Exception { + // Skip tests if DiskANN native library is not available + if (!DiskAnn.isLibraryLoaded()) { + try { + DiskAnn.loadLibrary(); + } catch (DiskAnnException e) { + StringBuilder errorMsg = new StringBuilder("DiskANN native library not available."); + errorMsg.append("\nError: ").append(e.getMessage()); + if (e.getCause() != null) { + errorMsg.append("\nCause: ").append(e.getCause().getMessage()); + } + errorMsg.append( + "\n\nTo run DiskANN tests, ensure the paimon-diskann-jni JAR" + + " with native libraries is available in the classpath."); + Assumptions.assumeTrue(false, errorMsg.toString()); + } + } + + Path tablePath = new Path(tempDir.toString()); + fileIO = new LocalFileIO(); + SchemaManager schemaManager = new SchemaManager(fileIO, tablePath); + + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column(vectorFieldName, new ArrayType(DataTypes.FLOAT())) + .option(CoreOptions.BUCKET.key(), "-1") + .option("vector.dim", "2") + .option("vector.metric", "L2") + .option("vector.diskann.index-type", "MEMORY") + .option("data-evolution.enabled", "true") + .option("row-tracking.enabled", "true") + .build(); + + TableSchema tableSchema = schemaManager.createTable(schema); + table = FileStoreTableFactory.create(fileIO, tablePath, tableSchema); + rowType = table.rowType(); + commitUser = UUID.randomUUID().toString(); + } + + @Test + public void testVectorIndexScanEndToEnd() throws Exception { + float[][] vectors = + new float[][] { + new float[] {1.0f, 0.0f}, new float[] {0.95f, 0.1f}, new float[] {0.1f, 0.95f}, + new float[] {0.98f, 0.05f}, new float[] {0.0f, 1.0f}, new float[] {0.05f, 0.98f} + }; + + writeVectors(vectors); + + List indexFiles = buildIndexManually(vectors); + + commitIndex(indexFiles); + + float[] queryVector = new float[] {0.85f, 0.15f}; + VectorSearch vectorSearch = new VectorSearch(queryVector, 2, vectorFieldName); + ReadBuilder readBuilder = table.newReadBuilder().withVectorSearch(vectorSearch); + TableScan scan = readBuilder.newScan(); + List ids = new ArrayList<>(); + readBuilder + .newRead() + .createReader(scan.plan()) + .forEachRemaining( + row -> { + ids.add(row.getInt(0)); + }); + // With L2 distance, the closest vectors to [0.85, 0.15] should be [0.95, 0.1] and [0.98, + // 0.05] + assertThat(ids).containsExactlyInAnyOrder(1, 3); + } + + @Test + public void testVectorIndexScanWithDifferentMetrics() throws Exception { + Path tablePath = new Path(tempDir.toString(), "inner_product"); + fileIO.mkdirs(tablePath); + SchemaManager schemaManager = new SchemaManager(fileIO, tablePath); + + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column(vectorFieldName, new ArrayType(DataTypes.FLOAT())) + .option(CoreOptions.BUCKET.key(), "-1") + .option("vector.dim", "2") + .option("vector.metric", "INNER_PRODUCT") + .option("vector.diskann.index-type", "MEMORY") + .option("data-evolution.enabled", "true") + .option("row-tracking.enabled", "true") + .build(); + + TableSchema tableSchema = schemaManager.createTable(schema); + FileStoreTable ipTable = FileStoreTableFactory.create(fileIO, tablePath, tableSchema); + String ipCommitUser = UUID.randomUUID().toString(); + + float[][] vectors = + new float[][] { + new float[] {1.0f, 0.0f}, + new float[] {0.707f, 0.707f}, + new float[] {0.0f, 1.0f}, + }; + + StreamTableWrite write = ipTable.newWrite(ipCommitUser); + for (int i = 0; i < vectors.length; i++) { + write.write(GenericRow.of(i, new GenericArray(vectors[i]))); + } + List messages = write.prepareCommit(false, 0); + StreamTableCommit commit = ipTable.newCommit(ipCommitUser); + commit.commit(0, messages); + write.close(); + + Options options = new Options(ipTable.options()); + DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); + Path indexDir = ipTable.store().pathFactory().indexPath(); + if (!fileIO.exists(indexDir)) { + fileIO.mkdirs(indexDir); + } + + GlobalIndexFileWriter fileWriter = + new GlobalIndexFileWriter() { + @Override + public String newFileName(String prefix) { + return prefix + "-" + UUID.randomUUID(); + } + + @Override + public PositionOutputStream newOutputStream(String fileName) + throws IOException { + return fileIO.newOutputStream(new Path(indexDir, fileName), false); + } + }; + + DiskAnnVectorGlobalIndexWriter indexWriter = + new DiskAnnVectorGlobalIndexWriter( + fileWriter, new ArrayType(DataTypes.FLOAT()), indexOptions); + for (float[] vec : vectors) { + indexWriter.write(vec); + } + + List entries = indexWriter.finish(); + List metas = new ArrayList<>(); + int fieldId = ipTable.rowType().getFieldIndex(vectorFieldName); + + for (ResultEntry entry : entries) { + long fileSize = fileIO.getFileSize(new Path(indexDir, entry.fileName())); + GlobalIndexMeta globalMeta = + new GlobalIndexMeta(0, vectors.length - 1, fieldId, null, entry.meta()); + + metas.add( + new IndexFileMeta( + DiskAnnVectorGlobalIndexerFactory.IDENTIFIER, + entry.fileName(), + fileSize, + entry.rowCount(), + globalMeta, + (String) null)); + } + + DataIncrement dataIncrement = DataIncrement.indexIncrement(metas); + CommitMessage message = + new CommitMessageImpl( + BinaryRow.EMPTY_ROW, + 0, + 1, + dataIncrement, + CompactIncrement.emptyIncrement()); + ipTable.newCommit(ipCommitUser).commit(1, Collections.singletonList(message)); + + float[] queryVector = new float[] {1.0f, 0.0f}; + VectorSearch vectorSearch = new VectorSearch(queryVector, 1, vectorFieldName); + ReadBuilder readBuilder = ipTable.newReadBuilder().withVectorSearch(vectorSearch); + TableScan scan = readBuilder.newScan(); + List ids = new ArrayList<>(); + readBuilder + .newRead() + .createReader(scan.plan()) + .forEachRemaining( + row -> { + ids.add(row.getInt(0)); + }); + assertThat(ids).containsExactly(0); + } + + private void writeVectors(float[][] vectors) throws Exception { + StreamTableWrite write = table.newWrite(commitUser); + for (int i = 0; i < vectors.length; i++) { + write.write(GenericRow.of(i, new GenericArray(vectors[i]))); + } + List messages = write.prepareCommit(false, 0); + StreamTableCommit commit = table.newCommit(commitUser); + commit.commit(0, messages); + write.close(); + } + + private List buildIndexManually(float[][] vectors) throws Exception { + Options options = new Options(table.options()); + DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); + Path indexDir = table.store().pathFactory().indexPath(); + if (!fileIO.exists(indexDir)) { + fileIO.mkdirs(indexDir); + } + + GlobalIndexFileWriter fileWriter = + new GlobalIndexFileWriter() { + @Override + public String newFileName(String prefix) { + return prefix + "-" + UUID.randomUUID(); + } + + @Override + public PositionOutputStream newOutputStream(String fileName) + throws IOException { + return fileIO.newOutputStream(new Path(indexDir, fileName), false); + } + }; + + DiskAnnVectorGlobalIndexWriter writer = + new DiskAnnVectorGlobalIndexWriter( + fileWriter, new ArrayType(DataTypes.FLOAT()), indexOptions); + for (float[] vec : vectors) { + writer.write(vec); + } + + List entries = writer.finish(); + + List metas = new ArrayList<>(); + int fieldId = rowType.getFieldIndex(vectorFieldName); + + for (ResultEntry entry : entries) { + long fileSize = fileIO.getFileSize(new Path(indexDir, entry.fileName())); + GlobalIndexMeta globalMeta = + new GlobalIndexMeta(0, vectors.length - 1, fieldId, null, entry.meta()); + + metas.add( + new IndexFileMeta( + DiskAnnVectorGlobalIndexerFactory.IDENTIFIER, + entry.fileName(), + fileSize, + entry.rowCount(), + globalMeta, + (String) null)); + } + return metas; + } + + private void commitIndex(List indexFiles) { + StreamTableCommit commit = table.newCommit(commitUser); + DataIncrement dataIncrement = DataIncrement.indexIncrement(indexFiles); + CommitMessage message = + new CommitMessageImpl( + BinaryRow.EMPTY_ROW, + 0, + 1, + dataIncrement, + CompactIncrement.emptyIncrement()); + commit.commit(1, Collections.singletonList(message)); + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexTest.java b/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexTest.java new file mode 100644 index 000000000000..43a7066a64ff --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexTest.java @@ -0,0 +1,486 @@ +/* + * 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.paimon.diskann.index; + +import org.apache.paimon.diskann.DiskAnn; +import org.apache.paimon.diskann.DiskAnnException; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.PositionOutputStream; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.globalindex.GlobalIndexIOMeta; +import org.apache.paimon.globalindex.GlobalIndexResult; +import org.apache.paimon.globalindex.ResultEntry; +import org.apache.paimon.globalindex.io.GlobalIndexFileReader; +import org.apache.paimon.globalindex.io.GlobalIndexFileWriter; +import org.apache.paimon.options.Options; +import org.apache.paimon.predicate.VectorSearch; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.FloatType; +import org.apache.paimon.utils.RoaringNavigableMap64; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assumptions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.UUID; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link DiskAnnVectorGlobalIndexWriter} and {@link DiskAnnVectorGlobalIndexReader}. */ +public class DiskAnnVectorGlobalIndexTest { + + @TempDir java.nio.file.Path tempDir; + + private FileIO fileIO; + private Path indexPath; + private DataType vectorType; + private final String fieldName = "vec"; + + @BeforeEach + public void setup() { + // Skip tests if DiskANN native library is not available + if (!DiskAnn.isLibraryLoaded()) { + try { + DiskAnn.loadLibrary(); + } catch (DiskAnnException e) { + StringBuilder errorMsg = new StringBuilder("DiskANN native library not available."); + errorMsg.append("\nError: ").append(e.getMessage()); + if (e.getCause() != null) { + errorMsg.append("\nCause: ").append(e.getCause().getMessage()); + } + errorMsg.append( + "\n\nTo run DiskANN tests, ensure the paimon-diskann-jni JAR" + + " with native libraries is available in the classpath."); + Assumptions.assumeTrue(false, errorMsg.toString()); + } + } + + fileIO = new LocalFileIO(); + indexPath = new Path(tempDir.toString()); + vectorType = new ArrayType(new FloatType()); + } + + @AfterEach + public void cleanup() throws IOException { + if (fileIO != null) { + fileIO.delete(indexPath, true); + } + } + + private GlobalIndexFileWriter createFileWriter(Path path) { + return new GlobalIndexFileWriter() { + @Override + public String newFileName(String prefix) { + return prefix + "-" + UUID.randomUUID(); + } + + @Override + public PositionOutputStream newOutputStream(String fileName) throws IOException { + return fileIO.newOutputStream(new Path(path, fileName), false); + } + }; + } + + private GlobalIndexFileReader createFileReader(Path path) { + return meta -> fileIO.newInputStream(new Path(path, meta.filePath())); + } + + @Test + public void testDifferentMetrics() throws IOException { + int dimension = 32; + int numVectors = 20; + + String[] metrics = {"L2", "INNER_PRODUCT", "COSINE"}; + + for (String metric : metrics) { + Options options = createDefaultOptions(dimension); + options.setString("vector.metric", metric); + DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); + Path metricIndexPath = new Path(indexPath, metric.toLowerCase()); + GlobalIndexFileWriter fileWriter = createFileWriter(metricIndexPath); + DiskAnnVectorGlobalIndexWriter writer = + new DiskAnnVectorGlobalIndexWriter(fileWriter, vectorType, indexOptions); + + List testVectors = generateRandomVectors(numVectors, dimension); + testVectors.forEach(writer::write); + + List results = writer.finish(); + assertThat(results).hasSize(1); + + ResultEntry result = results.get(0); + GlobalIndexFileReader fileReader = createFileReader(metricIndexPath); + List metas = new ArrayList<>(); + metas.add( + new GlobalIndexIOMeta( + new Path(metricIndexPath, result.fileName()), + fileIO.getFileSize(new Path(metricIndexPath, result.fileName())), + result.meta())); + + try (DiskAnnVectorGlobalIndexReader reader = + new DiskAnnVectorGlobalIndexReader( + fileReader, metas, vectorType, indexOptions)) { + VectorSearch vectorSearch = new VectorSearch(testVectors.get(0), 3, fieldName); + GlobalIndexResult searchResult = reader.visitVectorSearch(vectorSearch).get(); + assertThat(searchResult).isNotNull(); + } + } + } + + @Test + public void testDifferentIndexTypes() throws IOException { + int dimension = 32; + int numVectors = 100; + + String[] indexTypes = {"MEMORY"}; + + for (String indexType : indexTypes) { + Options options = createDefaultOptions(dimension); + options.setString("vector.diskann.index-type", indexType); + DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); + Path typeIndexPath = new Path(indexPath, indexType.toLowerCase()); + GlobalIndexFileWriter fileWriter = createFileWriter(typeIndexPath); + DiskAnnVectorGlobalIndexWriter writer = + new DiskAnnVectorGlobalIndexWriter(fileWriter, vectorType, indexOptions); + + List testVectors = generateRandomVectors(numVectors, dimension); + testVectors.forEach(writer::write); + + List results = writer.finish(); + assertThat(results).hasSize(1); + + ResultEntry result = results.get(0); + GlobalIndexFileReader fileReader = createFileReader(typeIndexPath); + List metas = new ArrayList<>(); + metas.add( + new GlobalIndexIOMeta( + new Path(typeIndexPath, result.fileName()), + fileIO.getFileSize(new Path(typeIndexPath, result.fileName())), + result.meta())); + + try (DiskAnnVectorGlobalIndexReader reader = + new DiskAnnVectorGlobalIndexReader( + fileReader, metas, vectorType, indexOptions)) { + VectorSearch vectorSearch = new VectorSearch(testVectors.get(0), 5, fieldName); + GlobalIndexResult searchResult = reader.visitVectorSearch(vectorSearch).get(); + assertThat(searchResult).isNotNull(); + } + } + } + + @Test + public void testDifferentDimensions() throws IOException { + int[] dimensions = {8, 32, 128, 256}; + + for (int dimension : dimensions) { + Options options = createDefaultOptions(dimension); + DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); + Path dimIndexPath = new Path(indexPath, "dim_" + dimension); + GlobalIndexFileWriter fileWriter = createFileWriter(dimIndexPath); + DiskAnnVectorGlobalIndexWriter writer = + new DiskAnnVectorGlobalIndexWriter(fileWriter, vectorType, indexOptions); + + int numVectors = 10; + List testVectors = generateRandomVectors(numVectors, dimension); + testVectors.forEach(writer::write); + + List results = writer.finish(); + assertThat(results).hasSize(1); + + ResultEntry result = results.get(0); + GlobalIndexFileReader fileReader = createFileReader(dimIndexPath); + List metas = new ArrayList<>(); + metas.add( + new GlobalIndexIOMeta( + new Path(dimIndexPath, result.fileName()), + fileIO.getFileSize(new Path(dimIndexPath, result.fileName())), + result.meta())); + + try (DiskAnnVectorGlobalIndexReader reader = + new DiskAnnVectorGlobalIndexReader( + fileReader, metas, vectorType, indexOptions)) { + VectorSearch vectorSearch = new VectorSearch(testVectors.get(0), 5, fieldName); + GlobalIndexResult searchResult = reader.visitVectorSearch(vectorSearch).get(); + assertThat(searchResult).isNotNull(); + } + } + } + + @Test + public void testDimensionMismatch() throws IOException { + Options options = createDefaultOptions(64); + + GlobalIndexFileWriter fileWriter = createFileWriter(indexPath); + DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); + DiskAnnVectorGlobalIndexWriter writer = + new DiskAnnVectorGlobalIndexWriter(fileWriter, vectorType, indexOptions); + + float[] wrongDimVector = new float[32]; + assertThatThrownBy(() -> writer.write(wrongDimVector)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("dimension mismatch"); + } + + @Test + public void testFloatVectorIndexEndToEnd() throws IOException { + int dimension = 2; + Options options = createDefaultOptions(dimension); + int sizePerIndex = 3; + options.setInteger("vector.size-per-index", sizePerIndex); + + float[][] vectors = + new float[][] { + new float[] {1.0f, 0.0f}, new float[] {0.95f, 0.1f}, new float[] {0.1f, 0.95f}, + new float[] {0.98f, 0.05f}, new float[] {0.0f, 1.0f}, new float[] {0.05f, 0.98f} + }; + + GlobalIndexFileWriter fileWriter = createFileWriter(indexPath); + DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); + DiskAnnVectorGlobalIndexWriter writer = + new DiskAnnVectorGlobalIndexWriter(fileWriter, vectorType, indexOptions); + Arrays.stream(vectors).forEach(writer::write); + + List results = writer.finish(); + assertThat(results).hasSize(2); + + GlobalIndexFileReader fileReader = createFileReader(indexPath); + List metas = new ArrayList<>(); + for (ResultEntry result : results) { + metas.add( + new GlobalIndexIOMeta( + new Path(indexPath, result.fileName()), + fileIO.getFileSize(new Path(indexPath, result.fileName())), + result.meta())); + } + + try (DiskAnnVectorGlobalIndexReader reader = + new DiskAnnVectorGlobalIndexReader(fileReader, metas, vectorType, indexOptions)) { + VectorSearch vectorSearch = new VectorSearch(vectors[0], 1, fieldName); + DiskAnnScoredGlobalIndexResult result = + (DiskAnnScoredGlobalIndexResult) reader.visitVectorSearch(vectorSearch).get(); + assertThat(result.results().getLongCardinality()).isEqualTo(1); + long expectedRowId = 0; + assertThat(containsRowId(result, expectedRowId)).isTrue(); + + // Test with filter + expectedRowId = 1; + RoaringNavigableMap64 filterResults = new RoaringNavigableMap64(); + filterResults.add(expectedRowId); + vectorSearch = + new VectorSearch(vectors[0], 1, fieldName).withIncludeRowIds(filterResults); + result = (DiskAnnScoredGlobalIndexResult) reader.visitVectorSearch(vectorSearch).get(); + assertThat(containsRowId(result, expectedRowId)).isTrue(); + + // Test with multiple results + float[] queryVector = new float[] {0.85f, 0.15f}; + vectorSearch = new VectorSearch(queryVector, 2, fieldName); + result = (DiskAnnScoredGlobalIndexResult) reader.visitVectorSearch(vectorSearch).get(); + assertThat(result.results().getLongCardinality()).isEqualTo(2); + } + } + + @Test + public void testInvalidTopK() { + assertThatThrownBy(() -> new VectorSearch(new float[] {0.1f}, 0, fieldName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Limit must be positive"); + } + + @Test + public void testMultipleIndexFiles() throws IOException { + int dimension = 32; + Options options = createDefaultOptions(dimension); + options.setInteger("vector.size-per-index", 5); + + GlobalIndexFileWriter fileWriter = createFileWriter(indexPath); + DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); + DiskAnnVectorGlobalIndexWriter writer = + new DiskAnnVectorGlobalIndexWriter(fileWriter, vectorType, indexOptions); + + int numVectors = 15; + List testVectors = generateRandomVectors(numVectors, dimension); + testVectors.forEach(writer::write); + + List results = writer.finish(); + assertThat(results).hasSize(3); + + GlobalIndexFileReader fileReader = createFileReader(indexPath); + List metas = new ArrayList<>(); + for (ResultEntry result : results) { + metas.add( + new GlobalIndexIOMeta( + new Path(indexPath, result.fileName()), + fileIO.getFileSize(new Path(indexPath, result.fileName())), + result.meta())); + } + + try (DiskAnnVectorGlobalIndexReader reader = + new DiskAnnVectorGlobalIndexReader(fileReader, metas, vectorType, indexOptions)) { + VectorSearch vectorSearch = new VectorSearch(testVectors.get(10), 3, fieldName); + GlobalIndexResult searchResult = reader.visitVectorSearch(vectorSearch).get(); + assertThat(searchResult).isNotNull(); + assertThat(searchResult.results().getLongCardinality()).isGreaterThan(0); + } + } + + @Test + public void testBatchWriteMultipleFiles() throws IOException { + int dimension = 8; + Options options = createDefaultOptions(dimension); + int sizePerIndex = 100; + options.setInteger("vector.size-per-index", sizePerIndex); + + GlobalIndexFileWriter fileWriter = createFileWriter(indexPath); + DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); + DiskAnnVectorGlobalIndexWriter writer = + new DiskAnnVectorGlobalIndexWriter(fileWriter, vectorType, indexOptions); + + int numVectors = 350; + List testVectors = generateRandomVectors(numVectors, dimension); + testVectors.forEach(writer::write); + + List results = writer.finish(); + assertThat(results).hasSize(4); + + GlobalIndexFileReader fileReader = createFileReader(indexPath); + List metas = new ArrayList<>(); + for (ResultEntry result : results) { + Path filePath = new Path(indexPath, result.fileName()); + assertThat(fileIO.exists(filePath)).isTrue(); + assertThat(fileIO.getFileSize(filePath)).isGreaterThan(0); + metas.add(new GlobalIndexIOMeta(filePath, fileIO.getFileSize(filePath), result.meta())); + } + + try (DiskAnnVectorGlobalIndexReader reader = + new DiskAnnVectorGlobalIndexReader(fileReader, metas, vectorType, indexOptions)) { + VectorSearch vectorSearch = new VectorSearch(testVectors.get(50), 3, fieldName); + GlobalIndexResult searchResult = reader.visitVectorSearch(vectorSearch).get(); + assertThat(searchResult).isNotNull(); + assertThat(searchResult.results().getLongCardinality()).isGreaterThan(0); + + vectorSearch = new VectorSearch(testVectors.get(150), 3, fieldName); + searchResult = reader.visitVectorSearch(vectorSearch).get(); + assertThat(searchResult).isNotNull(); + assertThat(searchResult.results().getLongCardinality()).isGreaterThan(0); + + vectorSearch = new VectorSearch(testVectors.get(320), 3, fieldName); + searchResult = reader.visitVectorSearch(vectorSearch).get(); + assertThat(searchResult).isNotNull(); + assertThat(searchResult.results().getLongCardinality()).isGreaterThan(0); + + vectorSearch = new VectorSearch(testVectors.get(200), 1, fieldName); + DiskAnnScoredGlobalIndexResult result = + (DiskAnnScoredGlobalIndexResult) reader.visitVectorSearch(vectorSearch).get(); + assertThat(containsRowId(result, 200)).isTrue(); + } + } + + @Test + public void testBatchWriteWithRemainder() throws IOException { + int dimension = 16; + Options options = createDefaultOptions(dimension); + int sizePerIndex = 50; + options.setInteger("vector.size-per-index", sizePerIndex); + + GlobalIndexFileWriter fileWriter = createFileWriter(indexPath); + DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); + DiskAnnVectorGlobalIndexWriter writer = + new DiskAnnVectorGlobalIndexWriter(fileWriter, vectorType, indexOptions); + + int numVectors = 73; + List testVectors = generateRandomVectors(numVectors, dimension); + testVectors.forEach(writer::write); + + List results = writer.finish(); + assertThat(results).hasSize(2); + + GlobalIndexFileReader fileReader = createFileReader(indexPath); + List metas = new ArrayList<>(); + for (ResultEntry result : results) { + metas.add( + new GlobalIndexIOMeta( + new Path(indexPath, result.fileName()), + fileIO.getFileSize(new Path(indexPath, result.fileName())), + result.meta())); + } + + try (DiskAnnVectorGlobalIndexReader reader = + new DiskAnnVectorGlobalIndexReader(fileReader, metas, vectorType, indexOptions)) { + VectorSearch vectorSearch = new VectorSearch(testVectors.get(60), 1, fieldName); + DiskAnnScoredGlobalIndexResult result = + (DiskAnnScoredGlobalIndexResult) reader.visitVectorSearch(vectorSearch).get(); + assertThat(result).isNotNull(); + assertThat(containsRowId(result, 60)).isTrue(); + + vectorSearch = new VectorSearch(testVectors.get(72), 1, fieldName); + result = (DiskAnnScoredGlobalIndexResult) reader.visitVectorSearch(vectorSearch).get(); + assertThat(result).isNotNull(); + assertThat(containsRowId(result, 72)).isTrue(); + } + } + + private Options createDefaultOptions(int dimension) { + Options options = new Options(); + options.setInteger("vector.dim", dimension); + options.setString("vector.metric", "L2"); + options.setString("vector.diskann.index-type", "MEMORY"); + options.setInteger("vector.diskann.max-degree", 64); + options.setInteger("vector.diskann.build-list-size", 100); + options.setInteger("vector.diskann.search-list-size", 100); + return options; + } + + private List generateRandomVectors(int count, int dimension) { + Random random = new Random(42); + List vectors = new ArrayList<>(); + for (int i = 0; i < count; i++) { + float[] vector = new float[dimension]; + for (int j = 0; j < dimension; j++) { + vector[j] = random.nextFloat() * 2 - 1; + } + float norm = 0; + for (float v : vector) { + norm += v * v; + } + norm = (float) Math.sqrt(norm); + if (norm > 0) { + for (int m = 0; m < vector.length; m++) { + vector[m] /= norm; + } + } + vectors.add(vector); + } + return vectors; + } + + private boolean containsRowId(GlobalIndexResult result, long rowId) { + List resultIds = new ArrayList<>(); + result.results().iterator().forEachRemaining(resultIds::add); + return resultIds.contains(rowId); + } +} diff --git a/paimon-diskann/paimon-diskann-jni/pom.xml b/paimon-diskann/paimon-diskann-jni/pom.xml new file mode 100644 index 000000000000..b655689e756a --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/pom.xml @@ -0,0 +1,100 @@ + + + + 4.0.0 + + + paimon-diskann + org.apache.paimon + 1.4-SNAPSHOT + + + paimon-diskann-jni + Paimon : DiskANN JNI + + + + org.slf4j + slf4j-api + + + + + org.junit.jupiter + junit-jupiter + test + + + + + + + + org.apache.maven.plugins + maven-resources-plugin + + + so + dylib + + + + + + + + + + release + + + + + org.apache.maven.plugins + maven-resources-plugin + + + copy-native-libs + prepare-package + + copy-resources + + + ${project.build.outputDirectory} + + + ${project.basedir}/src/main/resources + + **/*.so + **/*.dylib + + + + + + + + + + + + diff --git a/paimon-diskann/paimon-diskann-jni/scripts/build-native.sh b/paimon-diskann/paimon-diskann-jni/scripts/build-native.sh new file mode 100755 index 000000000000..026daa7cfc45 --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/scripts/build-native.sh @@ -0,0 +1,201 @@ +#!/bin/bash +# +# 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. + +set -e + +SCRIPT_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)" +PROJECT_DIR="$(dirname "$SCRIPT_DIR")" +NATIVE_DIR="$PROJECT_DIR/src/main/native" +BUILD_DIR="$PROJECT_DIR/build/native" + +# Parse arguments +CLEAN=false +RELEASE=true + +while [[ $# -gt 0 ]]; do + case $1 in + --clean) + CLEAN=true + shift + ;; + --debug) + RELEASE=false + shift + ;; + --help) + echo "Usage: $0 [options]" + echo "" + echo "Options:" + echo " --clean Clean build directory before building" + echo " --debug Build in debug mode (default: release)" + echo " --help Show this help message" + echo "" + echo "Environment variables:" + echo " RUST_TARGET Cargo target triple (e.g. aarch64-apple-darwin)" + echo " RUSTFLAGS Extra rustc flags" + echo " CARGO_FEATURES Extra cargo features (comma-separated)" + echo "" + echo "Example:" + echo " $0 --clean" + exit 0 + ;; + *) + echo "Unknown option: $1" + exit 1 + ;; + esac +done + +echo "================================================" +echo "Building Paimon DiskANN JNI - Native Library" +echo "================================================" +echo "Build mode: $([ "$RELEASE" = true ] && echo release || echo debug)" +echo "" + +if [ "$CLEAN" = true ]; then + echo "Cleaning build directory..." + rm -rf "$BUILD_DIR" +fi + +mkdir -p "$BUILD_DIR" +cd "$NATIVE_DIR" + +if [ ! -f "$NATIVE_DIR/Cargo.toml" ]; then + echo "ERROR: Cargo.toml not found in $NATIVE_DIR" + echo "Place the Rust JNI crate under src/main/native." + exit 1 +fi + +# Ensure $HOME/.cargo/bin is in PATH (rustup installs here). +if [ -d "$HOME/.cargo/bin" ]; then + export PATH="$HOME/.cargo/bin:$PATH" +fi + +# ---------- Check for required tools: rustup, rustc, cargo ---------- +# The rust-toolchain.toml in the native crate directory specifies Rust 1.90.0 +# which is required by diskann-vector v0.45.0 (uses unsigned_is_multiple_of, +# stabilised in Rust 1.87). +REQUIRED_RUST_VERSION="1.90.0" + +# 1. Ensure rustup is available; install if missing. +if ! command -v rustup &> /dev/null; then + echo "" + echo "rustup not found. Installing rustup with Rust $REQUIRED_RUST_VERSION..." + curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh -s -- -y --default-toolchain "$REQUIRED_RUST_VERSION" + export PATH="$HOME/.cargo/bin:$PATH" +fi + +# 2. Ensure the required toolchain (and its cargo/rustc) is installed. +echo "" +echo "Ensuring the required Rust toolchain ($REQUIRED_RUST_VERSION) is installed..." +if ! rustup run "$REQUIRED_RUST_VERSION" rustc --version &> /dev/null; then + echo "Toolchain $REQUIRED_RUST_VERSION not found. Installing..." + rustup toolchain install "$REQUIRED_RUST_VERSION" --profile minimal +fi + +# 3. Verify cargo is usable with the required toolchain. +if ! rustup run "$REQUIRED_RUST_VERSION" cargo --version &> /dev/null; then + echo "cargo not usable with toolchain $REQUIRED_RUST_VERSION. Re-installing..." + rustup toolchain uninstall "$REQUIRED_RUST_VERSION" 2>/dev/null || true + rustup toolchain install "$REQUIRED_RUST_VERSION" --profile minimal +fi + +echo " rustc: $(rustup run "$REQUIRED_RUST_VERSION" rustc --version)" +echo " cargo: $(rustup run "$REQUIRED_RUST_VERSION" cargo --version)" + +# Detect platform +OS=$(uname -s) +ARCH=$(uname -m) + +echo "Detected platform: $OS $ARCH" + +# Build with Cargo +echo "" +echo "Building with Cargo..." + +CARGO_ARGS=() +if [ "$RELEASE" = true ]; then + CARGO_ARGS+=(--release) +fi +if [ -n "$RUST_TARGET" ]; then + CARGO_ARGS+=(--target "$RUST_TARGET") + echo "Using RUST_TARGET: $RUST_TARGET" +fi +if [ -n "$CARGO_FEATURES" ]; then + CARGO_ARGS+=(--features "$CARGO_FEATURES") + echo "Using CARGO_FEATURES: $CARGO_FEATURES" +fi + +cargo build "${CARGO_ARGS[@]}" + +echo "" +echo "============================================" +echo "Build completed successfully!" +echo "============================================" + +# Determine output directory based on platform +if [ "$OS" = "Linux" ]; then + PLATFORM_OS="linux" + if [ "$ARCH" = "x86_64" ] || [ "$ARCH" = "amd64" ]; then + PLATFORM_ARCH="amd64" + else + PLATFORM_ARCH="aarch64" + fi +elif [ "$OS" = "Darwin" ]; then + PLATFORM_OS="darwin" + if [ "$ARCH" = "arm64" ]; then + PLATFORM_ARCH="aarch64" + else + PLATFORM_ARCH="amd64" + fi +else + echo "Unsupported OS: $OS" + exit 1 +fi + +OUTPUT_DIR="$PROJECT_DIR/src/main/resources/$PLATFORM_OS/$PLATFORM_ARCH" +mkdir -p "$OUTPUT_DIR" + +LIB_NAME="libpaimon_diskann_jni" +BUILD_MODE_DIR="$([ "$RELEASE" = true ] && echo release || echo debug)" + +if [ -n "$RUST_TARGET" ]; then + TARGET_DIR="target/$RUST_TARGET/$BUILD_MODE_DIR" +else + TARGET_DIR="target/$BUILD_MODE_DIR" +fi + +if [ "$OS" = "Darwin" ]; then + SRC_LIB="$NATIVE_DIR/$TARGET_DIR/$LIB_NAME.dylib" +else + SRC_LIB="$NATIVE_DIR/$TARGET_DIR/$LIB_NAME.so" +fi + +if [ ! -f "$SRC_LIB" ]; then + echo "ERROR: Built library not found: $SRC_LIB" + exit 1 +fi + +cp "$SRC_LIB" "$OUTPUT_DIR/" + +echo "" +echo "Native library location:" +ls -la "$OUTPUT_DIR/$(basename "$SRC_LIB")" +echo "" +echo "To package the JAR with native libraries, run:" +echo " mvn package" diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnn.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnn.java new file mode 100644 index 000000000000..dd252b82671c --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnn.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.diskann; + +/** Global DiskANN configuration and utilities. */ +public final class DiskAnn { + + static { + try { + NativeLibraryLoader.load(); + } catch (DiskAnnException e) { + // Library loading failed silently during class init. + // Callers should check isLibraryLoaded() or call loadLibrary() explicitly. + } + } + + private DiskAnn() {} + + /** + * Ensure the native library is loaded. + * + *

This method is called automatically when any DiskANN class is used. It can be called + * explicitly to load the library early and catch any loading errors. + * + * @throws DiskAnnException if the native library cannot be loaded + */ + public static void loadLibrary() throws DiskAnnException { + NativeLibraryLoader.load(); + } + + /** + * Check if the native library has been loaded. + * + * @return true if the library is loaded + */ + public static boolean isLibraryLoaded() { + return NativeLibraryLoader.isLoaded(); + } +} diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnException.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnException.java new file mode 100644 index 000000000000..21cbec85c41c --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnException.java @@ -0,0 +1,31 @@ +/* + * 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.paimon.diskann; + +/** Exception for DiskANN JNI failures. */ +public class DiskAnnException extends RuntimeException { + + public DiskAnnException(String message) { + super(message); + } + + public DiskAnnException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java new file mode 100644 index 000000000000..f4bdef48b125 --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java @@ -0,0 +1,91 @@ +/* + * 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.paimon.diskann; + +import java.nio.ByteBuffer; + +/** + * Native method declarations for DiskANN JNI with zero-copy support. + * + *

Users should not call these methods directly. Instead, use the high-level Java API classes + * like {@link Index}. + */ +final class DiskAnnNative { + + static { + try { + NativeLibraryLoader.load(); + } catch (DiskAnnException e) { + // Library loading failed silently during class init. + // Native methods will throw UnsatisfiedLinkError if called without the library. + } + } + + /** Create a DiskANN index with the given parameters. */ + static native long indexCreate( + int dimension, int metricType, int indexType, int maxDegree, int buildListSize); + + /** Destroy an index and free its resources. */ + static native void indexDestroy(long handle); + + /** Get the dimension of an index. */ + static native int indexGetDimension(long handle); + + /** Get the number of vectors in an index. */ + static native long indexGetCount(long handle); + + /** Get the metric type of an index. */ + static native int indexGetMetricType(long handle); + + /** Add vectors with IDs to an index using direct ByteBuffers (zero-copy). */ + static native void indexAddWithIds( + long handle, long n, ByteBuffer vectorBuffer, ByteBuffer idBuffer); + + /** Build the index graph after adding vectors. */ + static native void indexBuild(long handle, int buildListSize); + + /** + * Search for the k nearest neighbors. + * + * @param handle the native handle of the index + * @param n the number of query vectors + * @param queryVectors the query vectors (n * dimension floats) + * @param k the number of nearest neighbors to find + * @param searchListSize the size of the search list (DiskANN L parameter) + * @param distances output array for distances (n * k floats) + * @param labels output array for labels (n * k longs) + */ + static native void indexSearch( + long handle, + long n, + float[] queryVectors, + int k, + int searchListSize, + float[] distances, + long[] labels); + + /** Serialize an index to a direct ByteBuffer (zero-copy). */ + static native long indexSerialize(long handle, ByteBuffer buffer); + + /** Get the size in bytes needed to serialize an index. */ + static native long indexSerializeSize(long handle); + + /** Deserialize an index from a byte array. */ + static native long indexDeserialize(byte[] data, long length); +} diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java new file mode 100644 index 000000000000..6d5ddc059b67 --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java @@ -0,0 +1,165 @@ +/* + * 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.paimon.diskann; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * A DiskANN index for similarity search with zero-copy support. + * + *

Thread Safety: Index instances are NOT thread-safe. External synchronization is required if an + * index is accessed from multiple threads. + */ +public class Index implements AutoCloseable { + + /** Native handle to the DiskANN index. */ + private long nativeHandle; + + /** The dimension of vectors in this index. */ + private final int dimension; + + /** Whether this index has been closed. */ + private volatile boolean closed = false; + + Index(long nativeHandle, int dimension) { + this.nativeHandle = nativeHandle; + this.dimension = dimension; + } + + public int getDimension() { + return dimension; + } + + public long getCount() { + checkNotClosed(); + return DiskAnnNative.indexGetCount(nativeHandle); + } + + public MetricType getMetricType() { + checkNotClosed(); + return MetricType.fromValue(DiskAnnNative.indexGetMetricType(nativeHandle)); + } + + public void addWithIds(long n, ByteBuffer vectorBuffer, ByteBuffer idBuffer) { + checkNotClosed(); + validateDirectBuffer(vectorBuffer, n * dimension * Float.BYTES, "vector"); + validateDirectBuffer(idBuffer, n * Long.BYTES, "id"); + DiskAnnNative.indexAddWithIds(nativeHandle, n, vectorBuffer, idBuffer); + } + + public void build(int buildListSize) { + checkNotClosed(); + DiskAnnNative.indexBuild(nativeHandle, buildListSize); + } + + public void search( + long n, + float[] queryVectors, + int k, + int searchListSize, + float[] distances, + long[] labels) { + checkNotClosed(); + if (queryVectors.length < n * dimension) { + throw new IllegalArgumentException( + "Query vectors array too small: required " + + (n * dimension) + + ", got " + + queryVectors.length); + } + if (distances.length < n * k) { + throw new IllegalArgumentException( + "Distances array too small: required " + (n * k) + ", got " + distances.length); + } + if (labels.length < n * k) { + throw new IllegalArgumentException( + "Labels array too small: required " + (n * k) + ", got " + labels.length); + } + DiskAnnNative.indexSearch( + nativeHandle, n, queryVectors, k, searchListSize, distances, labels); + } + + public long serializeSize() { + checkNotClosed(); + return DiskAnnNative.indexSerializeSize(nativeHandle); + } + + public long serialize(ByteBuffer buffer) { + checkNotClosed(); + if (!buffer.isDirect()) { + throw new IllegalArgumentException("Buffer must be a direct buffer"); + } + return DiskAnnNative.indexSerialize(nativeHandle, buffer); + } + + public static Index deserialize(byte[] data) { + long handle = DiskAnnNative.indexDeserialize(data, data.length); + int dimension = DiskAnnNative.indexGetDimension(handle); + return new Index(handle, dimension); + } + + public static Index create( + int dimension, MetricType metricType, int indexType, int maxDegree, int buildListSize) { + long handle = + DiskAnnNative.indexCreate( + dimension, metricType.value(), indexType, maxDegree, buildListSize); + return new Index(handle, dimension); + } + + public static ByteBuffer allocateVectorBuffer(int numVectors, int dimension) { + return ByteBuffer.allocateDirect(numVectors * dimension * Float.BYTES) + .order(ByteOrder.nativeOrder()); + } + + public static ByteBuffer allocateIdBuffer(int numIds) { + return ByteBuffer.allocateDirect(numIds * Long.BYTES).order(ByteOrder.nativeOrder()); + } + + private void validateDirectBuffer(ByteBuffer buffer, long requiredBytes, String name) { + if (!buffer.isDirect()) { + throw new IllegalArgumentException(name + " buffer must be a direct buffer"); + } + if (buffer.capacity() < requiredBytes) { + throw new IllegalArgumentException( + name + + " buffer too small: required " + + requiredBytes + + " bytes, got " + + buffer.capacity()); + } + } + + private void checkNotClosed() { + if (closed) { + throw new IllegalStateException("Index has been closed"); + } + } + + @Override + public void close() { + if (!closed) { + closed = true; + if (nativeHandle != 0) { + DiskAnnNative.indexDestroy(nativeHandle); + nativeHandle = 0; + } + } + } +} diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/MetricType.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/MetricType.java new file mode 100644 index 000000000000..d995866a91d6 --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/MetricType.java @@ -0,0 +1,45 @@ +/* + * 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.paimon.diskann; + +/** DiskANN metric type mappings for JNI. */ +public enum MetricType { + L2(0), + INNER_PRODUCT(1), + COSINE(2); + + private final int value; + + MetricType(int value) { + this.value = value; + } + + public int value() { + return value; + } + + public static MetricType fromValue(int value) { + for (MetricType type : values()) { + if (type.value == value) { + return type; + } + } + throw new IllegalArgumentException("Unknown metric type value: " + value); + } +} diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/NativeLibraryLoader.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/NativeLibraryLoader.java new file mode 100644 index 000000000000..187ee244e482 --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/NativeLibraryLoader.java @@ -0,0 +1,256 @@ +/* + * 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.paimon.diskann; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.Path; + +/** + * Native library loader for DiskANN JNI. + * + *

The loader attempts to load the library in the following order: + * + *

    + *
  1. From the path specified by the {@code paimon.diskann.lib.path} system property + *
  2. From the system library path using {@code System.loadLibrary} + *
  3. From the JAR file bundled with the distribution + *
+ */ +public class NativeLibraryLoader { + private static final Logger LOG = LoggerFactory.getLogger(NativeLibraryLoader.class); + + /** The name of the native library. */ + private static final String JNI_LIBRARY_NAME = "paimon_diskann_jni"; + + /** System property to specify a custom path to the native library. */ + private static final String LIBRARY_PATH_PROPERTY = "paimon.diskann.lib.path"; + + /** Dependency libraries that need to be loaded before the main JNI library. */ + private static final String[] DEPENDENCY_LIBRARIES = {}; + + /** Whether the native library has been loaded. */ + private static volatile boolean libraryLoaded = false; + + /** Lock for thread-safe library loading. */ + private static final Object LOAD_LOCK = new Object(); + + /** Temporary directory for extracting native libraries. */ + private static Path tempDir; + + private NativeLibraryLoader() { + // Utility class, no instantiation + } + + /** + * Load the native library. + * + * @throws DiskAnnException if the library cannot be loaded + */ + public static void load() throws DiskAnnException { + if (libraryLoaded) { + return; + } + + synchronized (LOAD_LOCK) { + if (libraryLoaded) { + return; + } + + try { + loadNativeLibrary(); + libraryLoaded = true; + LOG.info("DiskANN native library loaded successfully"); + } catch (Exception e) { + throw new DiskAnnException("Failed to load DiskANN native library", e); + } + } + } + + /** + * Check if the native library has been loaded. + * + * @return true if the library is loaded + */ + public static boolean isLoaded() { + return libraryLoaded; + } + + private static void loadNativeLibrary() throws IOException { + // First, try loading from custom path + String customPath = System.getProperty(LIBRARY_PATH_PROPERTY); + if (customPath != null && !customPath.isEmpty()) { + File customLibrary = new File(customPath); + if (customLibrary.exists()) { + System.load(customLibrary.getAbsolutePath()); + LOG.info("Loaded DiskANN native library from custom path: {}", customPath); + return; + } else { + LOG.warn("Custom library path specified but file not found: {}", customPath); + } + } + + // Second, try loading from system library path + try { + System.loadLibrary(JNI_LIBRARY_NAME); + LOG.info("Loaded DiskANN native library from system path"); + return; + } catch (UnsatisfiedLinkError e) { + LOG.debug( + "Could not load from system path, trying bundled library: {}", e.getMessage()); + } + + // Third, try loading from JAR + loadFromJar(); + } + + private static void loadFromJar() throws IOException { + String libraryPath = getLibraryResourcePath(); + LOG.debug("Attempting to load native library from JAR: {}", libraryPath); + + try (InputStream is = NativeLibraryLoader.class.getResourceAsStream(libraryPath)) { + if (is == null) { + throw new IOException( + "Native library not found in JAR: " + + libraryPath + + ". " + + "Make sure you are using the correct JAR for your platform (" + + getPlatformIdentifier() + + ")"); + } + + // Create temp directory if needed + if (tempDir == null) { + tempDir = Files.createTempDirectory("paimon-diskann-native"); + tempDir.toFile().deleteOnExit(); + } + + // Extract and load dependency libraries (if bundled) + loadDependencyLibraries(); + + // Extract native library to temp file + String fileName = System.mapLibraryName(JNI_LIBRARY_NAME); + File tempFile = new File(tempDir.toFile(), fileName); + tempFile.deleteOnExit(); + + try (OutputStream os = new FileOutputStream(tempFile)) { + byte[] buffer = new byte[8192]; + int bytesRead; + while ((bytesRead = is.read(buffer)) != -1) { + os.write(buffer, 0, bytesRead); + } + } + + // Make the file executable (for Unix-like systems) + if (!tempFile.setExecutable(true)) { + LOG.warn("Could not set executable permission on native library"); + } + + // Load the library + System.load(tempFile.getAbsolutePath()); + LOG.info("Loaded DiskANN native library from JAR: {}", libraryPath); + } + } + + private static void loadDependencyLibraries() { + String os = getOsName(); + String arch = getArchName(); + + for (String depLib : DEPENDENCY_LIBRARIES) { + String resourcePath = "/" + os + "/" + arch + "/" + depLib; + try (InputStream is = NativeLibraryLoader.class.getResourceAsStream(resourcePath)) { + if (is == null) { + LOG.warn("Dependency library not bundled: {}", depLib); + continue; + } + + File tempFile = new File(tempDir.toFile(), depLib); + tempFile.deleteOnExit(); + + try (OutputStream fos = new FileOutputStream(tempFile)) { + byte[] buffer = new byte[8192]; + int bytesRead; + while ((bytesRead = is.read(buffer)) != -1) { + fos.write(buffer, 0, bytesRead); + } + } + + if (!tempFile.setExecutable(true)) { + LOG.warn("Could not set executable permission on: {}", depLib); + } + + System.load(tempFile.getAbsolutePath()); + LOG.info("Loaded bundled dependency library: {}", depLib); + } catch (UnsatisfiedLinkError e) { + LOG.warn("Could not load dependency {}: {}", depLib, e.getMessage()); + } catch (IOException e) { + LOG.warn("Could not extract dependency {}: {}", depLib, e.getMessage()); + } + } + } + + private static String getLibraryResourcePath() { + String os = getOsName(); + String arch = getArchName(); + String libraryFileName = System.mapLibraryName(JNI_LIBRARY_NAME); + return "/" + os + "/" + arch + "/" + libraryFileName; + } + + static String getPlatformIdentifier() { + return getOsName() + "/" + getArchName(); + } + + private static String getOsName() { + String osName = System.getProperty("os.name").toLowerCase(); + + if (osName.contains("linux")) { + return "linux"; + } else if (osName.contains("mac") || osName.contains("darwin")) { + return "darwin"; + } else { + throw new UnsupportedOperationException( + "Unsupported operating system: " + + osName + + ". Only Linux and macOS are supported."); + } + } + + private static String getArchName() { + String osArch = System.getProperty("os.arch").toLowerCase(); + + if (osArch.equals("amd64") || osArch.equals("x86_64")) { + return "amd64"; + } else if (osArch.equals("aarch64") || osArch.equals("arm64")) { + return "aarch64"; + } else { + throw new UnsupportedOperationException("Unsupported architecture: " + osArch); + } + } + + public static String getLibraryName() { + return JNI_LIBRARY_NAME; + } +} diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.lock b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.lock new file mode 100644 index 000000000000..170dc286eaa7 --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.lock @@ -0,0 +1,682 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +# 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. +# +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 4 + +[[package]] +name = "anyhow" +version = "1.0.101" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f0e0fee31ef5ed1ba1316088939cea399010ed7731dba877ed44aeb407a75ea" + +[[package]] +name = "autocfg" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" + +[[package]] +name = "bitflags" +version = "2.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "812e12b5285cc515a9c72a5c1d3b6d46a19dac5acfef5265968c166106e31dd3" + +[[package]] +name = "bytemuck" +version = "1.25.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8efb64bd706a16a1bdde310ae86b351e4d21550d98d056f22f8a7f7a2183fec" +dependencies = [ + "bytemuck_derive", +] + +[[package]] +name = "bytemuck_derive" +version = "1.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9abbd1bc6865053c427f7198e6af43bfdedc55ab791faed4fbd361d789575ff" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "bytes" +version = "1.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e748733b7cbc798e1434b6ac524f0c1ff2ab456fe201501e6497c8417a4fc33" + +[[package]] +name = "cesu8" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d43a04d8753f35258c91f8ec639f792891f748a1edbd759cf1dcea3382ad83c" + +[[package]] +name = "cfg-if" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9330f8b2ff13f34540b44e946ef35111825727b38d33286ef986142615121801" + +[[package]] +name = "combine" +version = "4.6.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba5a308b75df32fe02788e748662718f03fde005016435c444eea572398219fd" +dependencies = [ + "bytes", + "memchr", +] + +[[package]] +name = "crossbeam-utils" +version = "0.8.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" + +[[package]] +name = "crunchy" +version = "0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "460fbee9c2c2f33933d720630a6a0bac33ba7053db5344fac858d4b8952d77d5" + +[[package]] +name = "dashmap" +version = "6.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5041cc499144891f3790297212f32a74fb938e5136a14943f338ef9e0ae276cf" +dependencies = [ + "cfg-if", + "crossbeam-utils", + "hashbrown 0.14.5", + "lock_api", + "once_cell", + "parking_lot_core", +] + +[[package]] +name = "diskann" +version = "0.45.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52046f3e2d811ff08d458ddc649aaecddf2ba404c60086e556fb09436f3a9f4f" +dependencies = [ + "anyhow", + "bytemuck", + "dashmap", + "diskann-utils", + "diskann-vector", + "diskann-wide", + "futures-util", + "half", + "hashbrown 0.16.1", + "num-traits", + "rand", + "thiserror 2.0.18", + "tokio", + "tracing", +] + +[[package]] +name = "diskann-utils" +version = "0.45.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89752de1b587b64aedca61a53f92bf82903df17fe483413ae5cfc6d061fe2cd3" +dependencies = [ + "cfg-if", + "diskann-vector", + "diskann-wide", + "half", + "rand", + "thiserror 2.0.18", +] + +[[package]] +name = "diskann-vector" +version = "0.45.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e6e50f7003938f1572e8a8b91e81fd2693bd40413e8d477a2e440da278bca7c" +dependencies = [ + "cfg-if", + "diskann-wide", + "half", +] + +[[package]] +name = "diskann-wide" +version = "0.45.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8c4507634ff2929569ea3b60d8f60ce22ff4c3b04d5aca864b6732a9e3f03f99" +dependencies = [ + "cfg-if", + "half", +] + +[[package]] +name = "foldhash" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77ce24cb58228fbb8aa041425bb1050850ac19177686ea6e0f41a70416f56fdb" + +[[package]] +name = "futures-core" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05f29059c0c2090612e8d742178b0580d2dc940c837851ad723096f87af6663e" + +[[package]] +name = "futures-task" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f90f7dce0722e95104fcb095585910c0977252f286e354b5e3bd38902cd99988" + +[[package]] +name = "futures-util" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9fa08315bb612088cc391249efdc3bc77536f16c91f6cf495e6fbe85b20a4a81" +dependencies = [ + "futures-core", + "futures-task", + "pin-project-lite", + "pin-utils", +] + +[[package]] +name = "getrandom" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "899def5c37c4fd7b2664648c28120ecec138e4d395b459e5ca34f9cce2dd77fd" +dependencies = [ + "cfg-if", + "libc", + "r-efi", + "wasip2", +] + +[[package]] +name = "half" +version = "2.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ea2d84b969582b4b1864a92dc5d27cd2b77b622a8d79306834f1be5ba20d84b" +dependencies = [ + "bytemuck", + "cfg-if", + "crunchy", + "num-traits", + "rand", + "rand_distr", + "zerocopy", +] + +[[package]] +name = "hashbrown" +version = "0.14.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" + +[[package]] +name = "hashbrown" +version = "0.16.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "841d1cc9bed7f9236f321df977030373f4a4163ae1a7dbfe1a51a2c1a51d9100" +dependencies = [ + "foldhash", +] + +[[package]] +name = "jni" +version = "0.21.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a87aa2bb7d2af34197c04845522473242e1aa17c12f4935d5856491a7fb8c97" +dependencies = [ + "cesu8", + "cfg-if", + "combine", + "jni-sys", + "log", + "thiserror 1.0.69", + "walkdir", + "windows-sys 0.45.0", +] + +[[package]] +name = "jni-sys" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" + +[[package]] +name = "libc" +version = "0.2.181" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "459427e2af2b9c839b132acb702a1c654d95e10f8c326bfc2ad11310e458b1c5" + +[[package]] +name = "libm" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6d2cec3eae94f9f509c767b45932f1ada8350c4bdb85af2fcab4a3c14807981" + +[[package]] +name = "lock_api" +version = "0.4.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "224399e74b87b5f3557511d98dff8b14089b3dadafcab6bb93eab67d3aace965" +dependencies = [ + "scopeguard", +] + +[[package]] +name = "log" +version = "0.4.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e5032e24019045c762d3c0f28f5b6b8bbf38563a65908389bf7978758920897" + +[[package]] +name = "memchr" +version = "2.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8ca58f447f06ed17d5fc4043ce1b10dd205e060fb3ce5b979b8ed8e59ff3f79" + +[[package]] +name = "num-traits" +version = "0.2.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "071dfc062690e90b734c0b2273ce72ad0ffa95f0c74596bc250dcfd960262841" +dependencies = [ + "autocfg", + "libm", +] + +[[package]] +name = "once_cell" +version = "1.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42f5e15c9953c5e4ccceeb2e7382a716482c34515315f7b03532b8b4e8393d2d" + +[[package]] +name = "paimon_diskann_jni" +version = "0.1.0" +dependencies = [ + "diskann", + "diskann-vector", + "jni", + "tokio", +] + +[[package]] +name = "parking_lot_core" +version = "0.9.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2621685985a2ebf1c516881c026032ac7deafcda1a2c9b7850dc81e3dfcb64c1" +dependencies = [ + "cfg-if", + "libc", + "redox_syscall", + "smallvec", + "windows-link", +] + +[[package]] +name = "pin-project-lite" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b3cff922bd51709b605d9ead9aa71031d81447142d828eb4a6eba76fe619f9b" + +[[package]] +name = "pin-utils" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" + +[[package]] +name = "ppv-lite86" +version = "0.2.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85eae3c4ed2f50dcfe72643da4befc30deadb458a9b590d720cde2f2b1e97da9" +dependencies = [ + "zerocopy", +] + +[[package]] +name = "proc-macro2" +version = "1.0.106" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8fd00f0bb2e90d81d1044c2b32617f68fcb9fa3bb7640c23e9c748e53fb30934" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "quote" +version = "1.0.44" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "21b2ebcf727b7760c461f091f9f0f539b77b8e87f2fd88131e7f1b433b3cece4" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "r-efi" +version = "5.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69cdb34c158ceb288df11e18b4bd39de994f6657d83847bdffdbd7f346754b0f" + +[[package]] +name = "rand" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6db2770f06117d490610c7488547d543617b21bfa07796d7a12f6f1bd53850d1" +dependencies = [ + "rand_chacha", + "rand_core", +] + +[[package]] +name = "rand_chacha" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3022b5f1df60f26e1ffddd6c66e8aa15de382ae63b3a0c1bfc0e4d3e3f325cb" +dependencies = [ + "ppv-lite86", + "rand_core", +] + +[[package]] +name = "rand_core" +version = "0.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76afc826de14238e6e8c374ddcc1fa19e374fd8dd986b0d2af0d02377261d83c" +dependencies = [ + "getrandom", +] + +[[package]] +name = "rand_distr" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6a8615d50dcf34fa31f7ab52692afec947c4dd0ab803cc87cb3b0b4570ff7463" +dependencies = [ + "num-traits", + "rand", +] + +[[package]] +name = "redox_syscall" +version = "0.5.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed2bf2547551a7053d6fdfafda3f938979645c44812fbfcda098faae3f1a362d" +dependencies = [ + "bitflags", +] + +[[package]] +name = "same-file" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93fc1dc3aaa9bfed95e02e6eadabb4baf7e3078b0bd1b4d7b6b0b68378900502" +dependencies = [ + "winapi-util", +] + +[[package]] +name = "scopeguard" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" + +[[package]] +name = "smallvec" +version = "1.15.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67b1b7a3b5fe4f1376887184045fcf45c69e92af734b7aaddc05fb777b6fbd03" + +[[package]] +name = "syn" +version = "2.0.114" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d4d107df263a3013ef9b1879b0df87d706ff80f65a86ea879bd9c31f9b307c2a" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "thiserror" +version = "1.0.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6aaf5339b578ea85b50e080feb250a3e8ae8cfcdff9a461c9ec2904bc923f52" +dependencies = [ + "thiserror-impl 1.0.69", +] + +[[package]] +name = "thiserror" +version = "2.0.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4288b5bcbc7920c07a1149a35cf9590a2aa808e0bc1eafaade0b80947865fbc4" +dependencies = [ + "thiserror-impl 2.0.18", +] + +[[package]] +name = "thiserror-impl" +version = "1.0.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "thiserror-impl" +version = "2.0.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebc4ee7f67670e9b64d05fa4253e753e016c6c95ff35b89b7941d6b856dec1d5" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "tokio" +version = "1.49.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72a2903cd7736441aac9df9d7688bd0ce48edccaadf181c3b90be801e81d3d86" +dependencies = [ + "pin-project-lite", +] + +[[package]] +name = "tracing" +version = "0.1.44" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "63e71662fa4b2a2c3a26f570f037eb95bb1f85397f3cd8076caed2f026a6d100" +dependencies = [ + "pin-project-lite", + "tracing-attributes", + "tracing-core", +] + +[[package]] +name = "tracing-attributes" +version = "0.1.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7490cfa5ec963746568740651ac6781f701c9c5ea257c58e057f3ba8cf69e8da" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "tracing-core" +version = "0.1.36" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "db97caf9d906fbde555dd62fa95ddba9eecfd14cb388e4f491a66d74cd5fb79a" +dependencies = [ + "once_cell", +] + +[[package]] +name = "unicode-ident" +version = "1.0.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "537dd038a89878be9b64dd4bd1b260315c1bb94f4d784956b81e27a088d9a09e" + +[[package]] +name = "walkdir" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29790946404f91d9c5d06f9874efddea1dc06c5efe94541a7d6863108e3a5e4b" +dependencies = [ + "same-file", + "winapi-util", +] + +[[package]] +name = "wasip2" +version = "1.0.2+wasi-0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9517f9239f02c069db75e65f174b3da828fe5f5b945c4dd26bd25d89c03ebcf5" +dependencies = [ + "wit-bindgen", +] + +[[package]] +name = "winapi-util" +version = "0.1.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" +dependencies = [ + "windows-sys 0.61.2", +] + +[[package]] +name = "windows-link" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0805222e57f7521d6a62e36fa9163bc891acd422f971defe97d64e70d0a4fe5" + +[[package]] +name = "windows-sys" +version = "0.45.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75283be5efb2831d37ea142365f009c02ec203cd29a3ebecbc093d52315b66d0" +dependencies = [ + "windows-targets", +] + +[[package]] +name = "windows-sys" +version = "0.61.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ae137229bcbd6cdf0f7b80a31df61766145077ddf49416a728b02cb3921ff3fc" +dependencies = [ + "windows-link", +] + +[[package]] +name = "windows-targets" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e5180c00cd44c9b1c88adb3693291f1cd93605ded80c250a75d472756b4d071" +dependencies = [ + "windows_aarch64_gnullvm", + "windows_aarch64_msvc", + "windows_i686_gnu", + "windows_i686_msvc", + "windows_x86_64_gnu", + "windows_x86_64_gnullvm", + "windows_x86_64_msvc", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "597a5118570b68bc08d8d59125332c54f1ba9d9adeedeef5b99b02ba2b0698f8" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e08e8864a60f06ef0d0ff4ba04124db8b0fb3be5776a5cd47641e942e58c4d43" + +[[package]] +name = "windows_i686_gnu" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c61d927d8da41da96a81f029489353e68739737d3beca43145c8afec9a31a84f" + +[[package]] +name = "windows_i686_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44d840b6ec649f480a41c8d80f9c65108b92d89345dd94027bfe06ac444d1060" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8de912b8b8feb55c064867cf047dda097f92d51efad5b491dfb98f6bbb70cb36" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26d41b46a36d453748aedef1486d5c7a85db22e56aff34643984ea85514e94a3" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9aec5da331524158c6d1a4ac0ab1541149c0b9505fde06423b02f5ef0106b9f0" + +[[package]] +name = "wit-bindgen" +version = "0.51.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d7249219f66ced02969388cf2bb044a09756a083d0fab1e566056b04d9fbcaa5" + +[[package]] +name = "zerocopy" +version = "0.8.39" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "db6d35d663eadb6c932438e763b262fe1a70987f9ae936e60158176d710cae4a" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.8.39" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4122cd3169e94605190e77839c9a40d40ed048d305bfdc146e7df40ab0f3e517" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml new file mode 100644 index 000000000000..352815c18818 --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml @@ -0,0 +1,29 @@ +# 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] +name = "paimon_diskann_jni" +version = "0.1.0" +edition = "2021" + +[lib] +crate-type = ["cdylib"] + +[dependencies] +jni = "0.21" +diskann = { version = "0.45.0", features = ["testing"] } +diskann-vector = "0.45.0" +tokio = { version = "1", features = ["rt"] } diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/rust-toolchain.toml b/paimon-diskann/paimon-diskann-jni/src/main/native/rust-toolchain.toml new file mode 100644 index 000000000000..6e8dd640a2ef --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/rust-toolchain.toml @@ -0,0 +1,20 @@ +# 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. + +# Microsoft DiskANN v0.45.0 requires Rust >= 1.87 due to +# diskann-vector's use of unsigned_is_multiple_of (stabilised in 1.87). +[toolchain] +channel = "1.90.0" diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs new file mode 100644 index 000000000000..fbea9973e85c --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs @@ -0,0 +1,848 @@ +/* + * 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. + */ + +//! JNI bindings for Apache Paimon's DiskANN vector index. +//! +//! This module uses Microsoft's official `diskann` Rust crate (v0.45.0) +//! from to provide graph-based +//! approximate nearest neighbor search via JNI. + +use jni::objects::{JByteArray, JByteBuffer, JClass, JPrimitiveArray, ReleaseMode}; +use jni::sys::{jfloat, jint, jlong}; +use jni::JNIEnv; + +use std::collections::HashMap; +use std::sync::{Arc, Mutex, OnceLock}; + +use diskann::graph::test::provider as test_provider; +use diskann::graph::{self, DiskANNIndex}; +use diskann::neighbor::{BackInserter, Neighbor}; +use diskann_vector::distance::Metric; + +// ======================== Constants ======================== + +const METRIC_L2: i32 = 0; +const METRIC_INNER_PRODUCT: i32 = 1; +const METRIC_COSINE: i32 = 2; + +/// Serialization magic number ("PDAN"). +const MAGIC: i32 = 0x5044414E; +/// Serialization format version (2 = real DiskANN). +const SERIALIZE_VERSION: i32 = 2; + +/// The u32 ID reserved for the DiskANN graph start/entry point. +/// This is not a user vector and is filtered from search results. +const START_POINT_ID: u32 = 0; + +// ======================== Metric Mapping ======================== + +fn map_metric(metric_type: i32) -> Metric { + match metric_type { + METRIC_INNER_PRODUCT => Metric::InnerProduct, + METRIC_COSINE => Metric::Cosine, + _ => Metric::L2, + } +} + +// ======================== Index State ======================== + +/// Holds the DiskANN index, tokio runtime, and ID mappings. +struct IndexState { + /// The real DiskANN graph index backed by an in-memory test provider. + index: Arc>, + /// Execution context for the test provider. + context: test_provider::Context, + /// Tokio runtime for running async DiskANN operations. + runtime: tokio::runtime::Runtime, + + // -- Metadata -- + dimension: i32, + metric_type: i32, + index_type: i32, + max_degree: usize, + build_list_size: usize, + + // -- ID mapping (user i64 ↔ DiskANN u32) -- + ext_to_int: HashMap, + int_to_ext: HashMap, + /// Next u32 ID to assign (0 is reserved for start point). + next_id: u32, + + // -- Raw data kept for serialization -- + raw_data: Vec<(i64, Vec)>, +} + +// ======================== Registry ======================== + +struct IndexRegistry { + next_handle: i64, + indices: HashMap>>, +} + +impl IndexRegistry { + fn new() -> Self { + Self { + next_handle: 1, + indices: HashMap::new(), + } + } + + fn insert(&mut self, state: IndexState) -> i64 { + let handle = self.next_handle; + self.next_handle += 1; + self.indices.insert(handle, Arc::new(Mutex::new(state))); + handle + } +} + +fn registry() -> &'static Mutex { + static REGISTRY: OnceLock> = OnceLock::new(); + REGISTRY.get_or_init(|| Mutex::new(IndexRegistry::new())) +} + +/// Get a cloned Arc to the index state (brief registry lock). +fn get_index(handle: i64) -> Option>> { + let guard = registry().lock().ok()?; + guard.indices.get(&handle).cloned() +} + +// ======================== Index Construction ======================== + +/// Create a new DiskANN index backed by the official `diskann` crate. +fn create_index_state( + dimension: i32, + metric_type: i32, + index_type: i32, + max_degree: i32, + build_list_size: i32, +) -> Result { + let dim = dimension as usize; + let metric = map_metric(metric_type); + let md = std::cmp::max(max_degree as usize, 4); + // l_build must be >= pruned_degree for the DiskANN config to validate. + let bls = std::cmp::max(build_list_size as usize, md); + + // Tokio runtime for async DiskANN operations. + let runtime = tokio::runtime::Builder::new_current_thread() + .enable_all() + .build() + .map_err(|e| format!("Failed to create tokio runtime: {}", e))?; + + // The DiskANN graph needs at least one start/entry point. + // Use a non-zero vector to avoid division-by-zero with cosine metric. + let start_vector = vec![1.0f32; dim]; + let provider_config = test_provider::Config::new( + metric, + md, + test_provider::StartPoint::new(START_POINT_ID, start_vector), + ) + .map_err(|e| format!("Failed to create provider config: {:?}", e))?; + let provider = test_provider::Provider::new(provider_config); + + // Build graph config using the metric's default prune kind. + // Use MaxDegree::same() because the test provider enforces a strict max_degree + // and does not allow slack (the graph construction would exceed the provider limit). + let index_config = graph::config::Builder::new( + md, + graph::config::MaxDegree::same(), + bls, + metric.into(), // Metric → PruneKind conversion + ) + .build() + .map_err(|e| format!("Failed to create index config: {:?}", e))?; + + let index = Arc::new(DiskANNIndex::new(index_config, provider, None)); + let context = test_provider::Context::default(); + + Ok(IndexState { + index, + context, + runtime, + dimension, + metric_type, + index_type, + max_degree: md, + build_list_size: bls, + ext_to_int: HashMap::new(), + int_to_ext: HashMap::new(), + next_id: START_POINT_ID + 1, + raw_data: Vec::new(), + }) +} + +// ======================== Buffer Helpers ======================== + +fn get_direct_buffer_slice<'a>( + env: &mut JNIEnv, + buffer: &JByteBuffer, + len: usize, +) -> Option<&'a mut [u8]> { + let ptr = env.get_direct_buffer_address(buffer).ok()?; + let capacity = env.get_direct_buffer_capacity(buffer).ok()?; + if capacity < len { + return None; + } + // SAFETY: The caller guarantees the buffer is valid for `len` bytes. + unsafe { Some(std::slice::from_raw_parts_mut(ptr, len)) } +} + +// ======================== Serialization Helpers ======================== + +fn read_i32(buf: &[u8], offset: &mut usize) -> Option { + if *offset + 4 > buf.len() { + return None; + } + let mut bytes = [0u8; 4]; + bytes.copy_from_slice(&buf[*offset..*offset + 4]); + *offset += 4; + Some(i32::from_ne_bytes(bytes)) +} + +fn read_i64(buf: &[u8], offset: &mut usize) -> Option { + if *offset + 8 > buf.len() { + return None; + } + let mut bytes = [0u8; 8]; + bytes.copy_from_slice(&buf[*offset..*offset + 8]); + *offset += 8; + Some(i64::from_ne_bytes(bytes)) +} + +fn read_f32(buf: &[u8], offset: &mut usize) -> Option { + if *offset + 4 > buf.len() { + return None; + } + let mut bytes = [0u8; 4]; + bytes.copy_from_slice(&buf[*offset..*offset + 4]); + *offset += 4; + Some(f32::from_ne_bytes(bytes)) +} + +fn write_i32(buf: &mut [u8], offset: &mut usize, value: i32) -> bool { + if *offset + 4 > buf.len() { + return false; + } + buf[*offset..*offset + 4].copy_from_slice(&value.to_ne_bytes()); + *offset += 4; + true +} + +fn write_i64(buf: &mut [u8], offset: &mut usize, value: i64) -> bool { + if *offset + 8 > buf.len() { + return false; + } + buf[*offset..*offset + 8].copy_from_slice(&value.to_ne_bytes()); + *offset += 8; + true +} + +fn write_f32(buf: &mut [u8], offset: &mut usize, value: f32) -> bool { + if *offset + 4 > buf.len() { + return false; + } + buf[*offset..*offset + 4].copy_from_slice(&value.to_ne_bytes()); + *offset += 4; + true +} + +/// Calculate serialization size: +/// Header (32 bytes) + count * (8 bytes ID + dim*4 bytes vector) +fn serialization_size(dimension: i32, count: usize) -> usize { + 8 * 4 + count * (8 + (dimension as usize) * 4) +} + +// ======================== JNI Functions ======================== + +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreate<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + dimension: jint, + metric_type: jint, + index_type: jint, + max_degree: jint, + build_list_size: jint, +) -> jlong { + match create_index_state(dimension, metric_type, index_type, max_degree, build_list_size) { + Ok(state) => match registry().lock() { + Ok(mut guard) => guard.insert(state), + Err(_) => { + let _ = + env.throw_new("java/lang/IllegalStateException", "DiskANN registry error"); + 0 + } + }, + Err(msg) => { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!("Failed to create DiskANN index: {}", msg), + ); + 0 + } + } +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexDestroy<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + handle: jlong, +) { + if let Ok(mut guard) = registry().lock() { + guard.indices.remove(&handle); + } else { + let _ = env.throw_new("java/lang/IllegalStateException", "DiskANN registry error"); + } +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexGetDimension<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + handle: jlong, +) -> jint { + let arc = match get_index(handle) { + Some(a) => a, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); + return 0; + } + }; + let result = match arc.lock() { + Ok(state) => state.dimension, + Err(_) => { + let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); + 0 + } + }; + result +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexGetCount<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + handle: jlong, +) -> jlong { + let arc = match get_index(handle) { + Some(a) => a, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); + return 0; + } + }; + let result = match arc.lock() { + Ok(state) => state.raw_data.len() as jlong, + Err(_) => { + let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); + 0 + } + }; + result +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexGetMetricType<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + handle: jlong, +) -> jint { + let arc = match get_index(handle) { + Some(a) => a, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); + return 0; + } + }; + let result = match arc.lock() { + Ok(state) => state.metric_type, + Err(_) => { + let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); + 0 + } + }; + result +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexAddWithIds<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + handle: jlong, + n: jlong, + vector_buffer: JByteBuffer<'local>, + id_buffer: JByteBuffer<'local>, +) { + let arc = match get_index(handle) { + Some(a) => a, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); + return; + } + }; + let mut state = match arc.lock() { + Ok(s) => s, + Err(_) => { + let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); + return; + } + }; + + let num = n as usize; + let dimension = state.dimension as usize; + let vec_len = num * dimension * 4; + let id_len = num * 8; + + let vec_bytes = match get_direct_buffer_slice(&mut env, &vector_buffer, vec_len) { + Some(slice) => slice, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid vector buffer"); + return; + } + }; + let id_bytes = match get_direct_buffer_slice(&mut env, &id_buffer, id_len) { + Some(slice) => slice, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid id buffer"); + return; + } + }; + + // SAFETY: Reinterpret byte buffers as typed slices. + let vectors = + unsafe { std::slice::from_raw_parts(vec_bytes.as_ptr() as *const f32, num * dimension) }; + let ids = unsafe { std::slice::from_raw_parts(id_bytes.as_ptr() as *const i64, num) }; + + let strat = test_provider::Strategy::new(); + + for i in 0..num { + let ext_id = ids[i]; + let base = i * dimension; + let vector = vectors[base..base + dimension].to_vec(); + + // Assign a DiskANN-internal u32 ID. + let int_id = state.next_id; + state.next_id += 1; + state.ext_to_int.insert(ext_id, int_id); + state.int_to_ext.insert(int_id, ext_id); + state.raw_data.push((ext_id, vector.clone())); + + // Insert into the DiskANN graph index (async → block_on). + let result = state.runtime.block_on(state.index.insert( + strat, + &state.context, + &int_id, + vector.as_slice(), + )); + + if let Err(e) = result { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!("DiskANN insert failed for id {}: {}", ext_id, e), + ); + return; + } + } +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexBuild<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + handle: jlong, + _build_list_size: jint, +) { + // DiskANN builds the Vamana graph incrementally during insert. + // This function is a no-op because the graph is already built. + if get_index(handle).is_none() { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); + } +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearch<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + handle: jlong, + n: jlong, + query_vectors: JPrimitiveArray<'local, jfloat>, + k: jint, + search_list_size: jint, + distances: JPrimitiveArray<'local, jfloat>, + labels: JPrimitiveArray<'local, jlong>, +) { + let num = n as usize; + let top_k = k as usize; + + let arc = match get_index(handle) { + Some(a) => a, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); + return; + } + }; + + // Read query vectors into owned Vec (releases borrow on env). + let query: Vec = { + let query_elements = + match unsafe { env.get_array_elements(&query_vectors, ReleaseMode::NoCopyBack) } { + Ok(arr) => arr, + Err(_) => { + let _ = env.throw_new( + "java/lang/IllegalArgumentException", + "Invalid query vectors", + ); + return; + } + }; + query_elements.iter().copied().collect() + }; + + let state = match arc.lock() { + Ok(s) => s, + Err(_) => { + let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); + return; + } + }; + + let dimension = state.dimension as usize; + let total_results = num * top_k; + let mut result_distances = vec![f32::MAX; total_results]; + let mut result_labels = vec![-1i64; total_results]; + + // If the index has no user vectors, return empty results. + if state.raw_data.is_empty() { + // Leave default values (distances=MAX, labels=-1). + } else { + let strat = test_provider::Strategy::new(); + + for qi in 0..num { + let query_vec = &query[qi * dimension..(qi + 1) * dimension]; + + // Request k+1 results to account for the start point being returned. + let search_k = top_k + 1; + let l_value = std::cmp::max(search_list_size as usize, search_k); + + let params = match graph::SearchParams::new(search_k, l_value, None) { + Ok(p) => p, + Err(e) => { + let _ = env.throw_new( + "java/lang/IllegalArgumentException", + format!("Invalid search params: {}", e), + ); + return; + } + }; + + let mut neighbors = vec![Neighbor::::default(); search_k]; + let search_result = state.runtime.block_on(state.index.search( + &strat, + &state.context, + query_vec, + ¶ms, + &mut BackInserter::new(&mut neighbors), + )); + + let stats = match search_result { + Ok(s) => s, + Err(e) => { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!("DiskANN search failed: {}", e), + ); + return; + } + }; + + // Collect results, filtering out the start point. + let result_count = stats.result_count as usize; + let mut count = 0; + for ri in 0..result_count { + if count >= top_k { + break; + } + let neighbor = &neighbors[ri]; + if neighbor.id == START_POINT_ID { + continue; // Skip the graph entry point. + } + let idx = qi * top_k + count; + result_labels[idx] = + *state.int_to_ext.get(&neighbor.id).unwrap_or(&(neighbor.id as i64)); + result_distances[idx] = neighbor.distance; + count += 1; + } + } + } + + // Release the index lock before writing results to JNI arrays. + drop(state); + + // Write distances back to the Java array. + { + let mut dist_elements = + match unsafe { env.get_array_elements(&distances, ReleaseMode::CopyBack) } { + Ok(arr) => arr, + Err(_) => { + let _ = + env.throw_new("java/lang/IllegalArgumentException", "Invalid distances"); + return; + } + }; + for i in 0..std::cmp::min(dist_elements.len(), result_distances.len()) { + dist_elements[i] = result_distances[i]; + } + } + + // Write labels back to the Java array. + { + let mut label_elements = + match unsafe { env.get_array_elements(&labels, ReleaseMode::CopyBack) } { + Ok(arr) => arr, + Err(_) => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid labels"); + return; + } + }; + for i in 0..std::cmp::min(label_elements.len(), result_labels.len()) { + label_elements[i] = result_labels[i]; + } + } +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSerialize<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + handle: jlong, + buffer: JByteBuffer<'local>, +) -> jlong { + let arc = match get_index(handle) { + Some(a) => a, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); + return 0; + } + }; + let state = match arc.lock() { + Ok(s) => s, + Err(_) => { + let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); + return 0; + } + }; + + let count = state.raw_data.len(); + let required = serialization_size(state.dimension, count); + + let buf = match get_direct_buffer_slice(&mut env, &buffer, required) { + Some(slice) => slice, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Buffer too small"); + return 0; + } + }; + + let mut offset = 0usize; + if !write_i32(buf, &mut offset, MAGIC) + || !write_i32(buf, &mut offset, SERIALIZE_VERSION) + || !write_i32(buf, &mut offset, state.dimension) + || !write_i32(buf, &mut offset, state.metric_type) + || !write_i32(buf, &mut offset, state.index_type) + || !write_i32(buf, &mut offset, state.max_degree as i32) + || !write_i32(buf, &mut offset, state.build_list_size as i32) + || !write_i32(buf, &mut offset, count as i32) + { + let _ = env.throw_new("java/lang/IllegalStateException", "Serialize header failed"); + return 0; + } + + for (id, vector) in &state.raw_data { + if !write_i64(buf, &mut offset, *id) { + let _ = env.throw_new("java/lang/IllegalStateException", "Serialize failed"); + return 0; + } + for &v in vector { + if !write_f32(buf, &mut offset, v) { + let _ = env.throw_new("java/lang/IllegalStateException", "Serialize failed"); + return 0; + } + } + } + + required as jlong +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSerializeSize<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + handle: jlong, +) -> jlong { + let arc = match get_index(handle) { + Some(a) => a, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); + return 0; + } + }; + let result = match arc.lock() { + Ok(state) => serialization_size(state.dimension, state.raw_data.len()) as jlong, + Err(_) => { + let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); + 0 + } + }; + result +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexDeserialize<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + data: JByteArray<'local>, + _length: jlong, +) -> jlong { + let bytes = match env.convert_byte_array(&data) { + Ok(data) => data, + Err(_) => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid data"); + return 0; + } + }; + + let mut offset = 0usize; + + // Read and validate header. + let magic = match read_i32(&bytes, &mut offset) { + Some(v) => v, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Data too short"); + return 0; + } + }; + if magic != MAGIC { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid magic number"); + return 0; + } + + let version = match read_i32(&bytes, &mut offset) { + Some(v) => v, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Data too short"); + return 0; + } + }; + + let dimension = read_i32(&bytes, &mut offset).unwrap_or(0); + let metric_type = read_i32(&bytes, &mut offset).unwrap_or(METRIC_L2); + let index_type = read_i32(&bytes, &mut offset).unwrap_or(0); + + // Version 2 includes max_degree and build_list_size; version 1 uses defaults. + let (max_degree, build_list_size, count) = if version >= 2 { + let md = read_i32(&bytes, &mut offset).unwrap_or(64); + let bls = read_i32(&bytes, &mut offset).unwrap_or(100); + let cnt = read_i32(&bytes, &mut offset).unwrap_or(0) as usize; + (md, bls, cnt) + } else if version == 1 { + // Legacy format: no max_degree/build_list_size fields. + let cnt = read_i32(&bytes, &mut offset).unwrap_or(0) as usize; + (64, 100, cnt) + } else { + let _ = env.throw_new( + "java/lang/IllegalArgumentException", + format!("Unsupported version: {}", version), + ); + return 0; + }; + + // Read vector data. + let dim = dimension as usize; + let mut entries: Vec<(i64, Vec)> = Vec::with_capacity(count); + for _ in 0..count { + let id = match read_i64(&bytes, &mut offset) { + Some(v) => v, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated data"); + return 0; + } + }; + let mut vector = Vec::with_capacity(dim); + for _ in 0..dim { + let v = match read_f32(&bytes, &mut offset) { + Some(val) => val, + None => { + let _ = + env.throw_new("java/lang/IllegalArgumentException", "Truncated data"); + return 0; + } + }; + vector.push(v); + } + entries.push((id, vector)); + } + + // Create a new DiskANN index and re-insert all vectors to rebuild the graph. + let mut state = + match create_index_state(dimension, metric_type, index_type, max_degree, build_list_size) { + Ok(s) => s, + Err(msg) => { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!("Failed to create index during deserialization: {}", msg), + ); + return 0; + } + }; + + let strat = test_provider::Strategy::new(); + for (ext_id, vector) in &entries { + let int_id = state.next_id; + state.next_id += 1; + state.ext_to_int.insert(*ext_id, int_id); + state.int_to_ext.insert(int_id, *ext_id); + state.raw_data.push((*ext_id, vector.clone())); + + let result = state.runtime.block_on(state.index.insert( + strat, + &state.context, + &int_id, + vector.as_slice(), + )); + + if let Err(e) = result { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!( + "Failed to re-insert vector {} during deserialization: {}", + ext_id, e + ), + ); + return 0; + } + } + + match registry().lock() { + Ok(mut guard) => guard.insert(state), + Err(_) => { + let _ = env.throw_new("java/lang/IllegalStateException", "DiskANN registry error"); + 0 + } + } +} diff --git a/paimon-diskann/paimon-diskann-jni/src/main/resources/darwin/aarch64/libpaimon_diskann_jni.dylib b/paimon-diskann/paimon-diskann-jni/src/main/resources/darwin/aarch64/libpaimon_diskann_jni.dylib new file mode 100755 index 0000000000000000000000000000000000000000..8cf7312b036a9420095d62ca83e26962cd5c0e57 GIT binary patch literal 1023296 zcmeF434B!5+5hjI31qT?kU$bplMpEhsHn&?0%ayq30qN!Sha0}5t#sr8WfN!6H?2B z(84WI1niq2ZOufjRH0@{twFkA^;Ix#t+q~}trKuV)?v&4``m>LaadH`{(bZLOzu7B zo_p5kJkN8U^PJy(`QZCU6BR{O`0L2;EPf9<6y-i;sYEI1{5IPZrKqUzitiTcQOEyt zB}5H(AXa{4rT-Nb&6+Z2mKosEKEIHeFmsY|BVmed`-@S$Z2@_|Y}TDOmK}>~pP!@1 z$mdPFkwn6g`8mw|XuJMrcM#J)zu=8Vejl0nNjNe;H zpEvoGMR8)HF%`GZw zJIkz3?Y+iJ5{}F-A}jq?+Bt{H8tsgD+Gx&Gml@YCY_&7id|XsCV{%d1+*@v(Iiu*d zS$CW;zoph7l3!%KBp;)LjSP#5Zkjx6vY9!T_Vru(oKe4z;*B&Cj?Ay^P2&!J;<;nW zZUE^@^-p65%?2Y3<_sUsE>a zj0T?kMSh%jD8q z``*FvbHhiHZ@{BbA&EiOP4G1l~lR6D89`+T=}5RQfSgn_o;+f@^JxjM`QF zw_ZYA zCO@OEDZ$o5zt=}WeJQ3d&yMjd?q!sQ@RPiZDK0ZQR_T9d*&Wh5@+(Dz+;!n4k=+vI zy|JhP>T&N)oLe?)$}Jb>MZWkga+mVwBqb^i+Ma8MrN89w95cN3p3chB3_a>o{c|X; zaGdhdD;a9s&PKYnCD#3<+{dK54we&cv=_auxD>7Rf0Ps@#uZlg-rKZ5@lNRUGbP25 zlIL_JDOYE7Q*M9QrWEcO{|En(d8+?NzM}26EA@^{R~VXISNyHE_mrP0)t>HVeAl-4 zgPKxZ{G6g4ndkQJ8UI|}##Ks+l>Lc3FCXoc@{&}gFykEM_7`KgrYpDC+Kq5#tWr2f zQBpFJi0edL7v=V#TzSXM)${Ner^IDZU(b)n8TId=+#aH>o(J4UeOJ1VwewZ#<+t9X8{`Mg)#^LlyDX7+1G`qixGFC@OxC;R(!_xHktZQ+v7;Q1>q^)|N-P>00D^AeP5z?OLY#ORdH9SPmPudl`Iid9HAdjdAR9{5U3&29BBHdC3*dP?cRW1~FYm9hACRV=$Te6+hfj`|#)F zbD3Grpte3byp(w8Oz*o0KCzGZ;v3at{NS0DN=n?N-U&f-{2bQ!Q8)eFE8FsXSMrQf zRt=|4Nes~YK3QmzKuS1 zoMo1g()M26iOPV_>-{8Um}C5N>8B{XVY$TjFw;ly22J7_qmc0Q2;N||xstN2dzqW9 z6}GJhiaPrDT^kEecOV<$kPS9%uiaL^&#uJ$Pwfln|Sa5p>Ya*ua6SPcKW-!olh@P=GJXLdx(Gc*+W8hRSRxXv}WYh4d*G@L3}$*3c` zLyCv`d2CAH$h=Xsy?{O#=|50(rJp|0q;G_Ftny^ONO@f;Pu`u?)mAOfBl8s6`UA9j z3(s1z1GgNWsrnC2wUMWtykjV%1O404f22ayS}Qtl-A(`QCyzb!?*-J^H3=GCsQQE1 zO8B?iZCgXzRn2jAtQP98g!gbCizaJp4%8rSeZ=0&Mtm)|Y zEwE{!eoEMvXV*dllyK$44sDSwUi0O}OFYlxv`_}mpNl8G&F{!I#;922G<__5p{^ul zh`%glNND>wef%EdS>1KYz?O=U{?MbN{5!98g@b(({K1ON!-Dq3!$SR%{Gq|nU;j>i zn{xC2wA$mGB5Q;8wEAX6S(}v^tKG?X=g{|#Y}M&Vc01!z-Oj$yOMDgM-mbjI54RTT zV>?MH4B1CHrCy>>Y%Wv=2K$WhZ!2LA&{tWis=bEZQdd!U;CB_*9B5q7ILt5c zT(DqR2;R7H!E?jPZI#19^OSI~V#%=Ps7vhs_p4`7-xBm7>N}2m{n&X=WJy2hrYg!;I{!mC^QsPOH$oyusX@L+%9`n$rJ?#5vj?{?D{ zQhsQe!3%;*+W5>*moPUf%=r0j@jl|cc{c6p6-xC`#=*%i%s6?sDcYE$t;-KXV|(pA z>ciWUoOLCIZS!rXZR>=8Bi|F0+c)OPTuZHA@w8HX2Xig7ivD`n=&#*HjQ>nmIGwpx ze4|}^XQfhoXn-pmsJPVsyNYfHRts(0d%~BK>)&0dRPWAE!!5kCXU^TphvDz_|D|33s2;R$A!|#t&!>uLp&N%vEgsR-( z=vK2bG)4`3sDrNqW0~&`E0gUS@=2ewtYlvC9KENhxeqe)GG+bUAK%jy+I{bnMRv!| zI;T>2CH)hm{pIw_FzG|)m6!Ww;_8Md>vm66!=Y;_^9D8ChjABr)5n0m-n^%Eyml4iO)Ec&X@RkIr!dTN|A#Ir+Bxx)vi=LfPVFVu%Ip`5tf%l!%I zuJC*{e&@7{7p)w@)z%UEb1PvMI4PcQGgfmiYbc0SG*6xq-oZN|dwhLd(&Y!n_I8EG z=VfTVzZMemT-F{kLr*PKSo(K|AxIrwXpA(9lB0ab%*ib*36#M1DKU2+!u1pXdw~@lJ`+ z|E_HGj4NCX5*M0B|19B}0i7=+%@}*2bwHA{7P>B95U=IgmFlM|3Wv<^s@%Sswu+ny zQHGDJk2=opL_B;n$osovU18zr_Mbp!*p~G*Zi?284YH$GUWDbSLm;`P0q`#h8kX*mvDEb5COg77LywN+ zD0C8nAJ64_1HVW<+}zY!SWCGNcfQk~hfE7W-%1Dcnm9wR3Ci51OsU7>?@|ZCaqqhV zxgC@)aqlaE5M$SJgInh*SqEDS1LRegZ`qyJ7@cK`iU4@ z_pa8$H)a0GTzEtcA1t#6LOq!WPpIKTYv5~75jUOr@KloXcbn20N}*SgpP8!_ZFQ5n zbbpO5 z`RS*hd2F5o*c5hQx;1E=@^+Wy2f|Od!#R{`^~pAP_I6~A=%o%<=D9iVy8Mc2%(+__ zBMd}?FC+bRlwH7e1pG$ktj_<1=M*Ps&)lX|7f=sJQhHUgsstPzlxK3_l?4gV^CR@< zqeg#D-rrhy9(|+t8TFBQCw<$NmqQyY{c68~y!1pP3%BRLX|rYlqa8 zTs!Cahx3g2UvaHI|8s(^>tm?51G!lGfh&-sIubk|Kr2E!<*q<{iz_fnjZ27YAr6^2 zP38t=E!(eFujai4^iy7Kyyoa^t7gk3rGUEn(5pQ*bRFcAN;`|uEzo&cD_)5ndDt4rWAch~wDJz#3B7rC zInVU>a;*2dqJN`%Q~sV#w(4O#*YooxYafz+2w|DatC`DQ(ajqZvPSMr&_{f+z>>tlKEhX+gC z_u9n`j1m@o$f}dPC;f~~KIPl|#+uJ8n#}5K(BzUst;_E+pILRV@)&i3`0I$b@yj{E z^T>Kgo2>k>iR<@~{7cG|x;8R4GH*ig4gCu5h{g+6(0}@Kc!Ah;WFG!zpIW_(dH6Ck zDP>81vgWeNm9WT!&DcmHW%rNCi=9Y4oi{I5yZK=2a;XpdLC3wTmxm0H^{C>G)qYN& ziSEDhKIBUj8GIZ0Mw7wQB);O4GWaIqMgCs!fY~;4Es{eydf8FdAs1~UuXeoQY|__Z zE3x=O5Zl7vq1#IO#m8?UzaR5EJo9((9Tom#gAdu^M=`9+S)1pxF1MexbwBHJz3-4Y zBiAX_Vgqr-qW2RYgts=+cH0lV6H=HXjUALNk!x^S&pTPmUdC7+P?V)<*{t8A+CEaT zeWcYtxlpY>i0vbU?V|`QdJL&uqi$`cyen+ty2Eut zMBggcB{>T;{Wct@`SA6YBd%>MM4WZ_)pH zdg}Hu=_BhV_MlrCuX9M#YPWAaJhLNpi=~bZ>KaF#;~A#}|B+2LZGSua)IXlWZWsrB zFGOaLH~O)^HB||>A0H|EbsKYzUx@KeS6xqD#&}s{CF|i6+p5y)uVlP7MA)i+$9RQ? zs?L@XsL&^WDyxWaJq0{P!}?hhl5NHn5Sf^w~oZ zal)rXUW*7tr4$p#^MUcKb0K7`taFx~IvoCQ4($Lf9 zysr9Z_f-P(pH!Su577lyBU?OAx}4}=184Vj1?If&_V4e5Zn6gZ1Ha~FiuTdnO7%Y0 zcimb4@24-6d`0Vezfzr!j&tCjdH!c9qYfHqwXx=BKWf{)dzfv;R;lCRT^4$yBjJyDghsTo+EwyiuCdE z_Qiu!uQ+C7MBm<$RSUh^vF|}Ei$7AUTkLLUDEab+P-ogS(iI5pRvJX_>PvppD0c*N zUUXC+Ve~8~^_%b?#(hU;S6~NgnH{@b4Xk4uM$lJ5bbQvifgKsdm$@5u>{c6O{Va1$ zr7yNoW(Dv6h~G!=snr46^o;(zdwtKnYW1JE7oA4v7F}Qq`=MHxu@(B%!@-W)Uc%OW zt94Jh!`zF`AbN(#OL<4eJhDGGn!HP)Ea4C5@{Xh_pgz*a`jtKuyY5tI%L?BlJR-jX zJfbr+dlod-1sd)OEhobxQv65k9oXMAY3wY+t~r+Vs4vZ=u}(_(t~^&@Htp>Roz0tM z^Uv-<+-1<($BOgdc2(QgA00N|ppljiO5wHHc7I)oZR_)PWoho@%d~<%F@DkS3;J~M zV`tjpD38%bQSZ}!I6Rn)O?oJEyPpz9f83I% zbkIVZM6Rfsi}%x^fxVkt&_I4eT}65W_S8VZfQ}7^pyLwyNS_0&?L8ke2f8Q&@4Czt zm^Uep`n$sI>+NCur5?p=$p0xtyLcWl78$=e23vEs;xFvy3fIu?!R(QChaL_?4;?9E z$h<)fuXj|m2YB{StgGGJu54@^ zrZjqo`A8e8i1mj?nth)cqsjO*Q=atw@1(p6nSWOr}KZw!Bz0?@qXro z&bxJ4^s!p~diyzC+(C0N&qQ|X^R_lti?QY{_noYJ(j{;&^q|veX0nz^o670KpClM_ zTX=FOexhf~dO>tjnd>UQ0<%B#EA^dCUe^6A?k!%>0v(JvV)Sb>{n7qCnfo5n2TAw# zF17jq?MX=#9U*pWOLjN*S-R8qRQe!|zDTD}dcXsEq9=6J4#NX-E@zDG9sN0`UqbT; zCHxrUC-OHjjy<}olz}z**j=#K^k5&shfSp#oACoPNBdWfSHnJRJJr~D9>BJ^3j1G; zJ;7Np5_>1Mg2~BBVY!WQ%eQMT=2+carF!plSNOFwWu3^H+57Kp^0eODc^iW%@>@uir2 zvs(B-a+2mtHueq@**hp0s|pXznW)-#}i_U`YwH6*)lVsTA znv{UfCo?;0brsl`puL}efb6zU8d8@$X^7;xK6}!f-{ntgSXVLW!2LoyjKRjCO5y$6 z8NcoL*C5l+M;_)3R|fhhvy$~;%?4Y*hrK%t4OU_=7aOv>P$~Qy`%$_d1GiSOfx5OQd#SU@l@%8mGe=4im>bziy zJAAf7Nm;%d+TLx>wcgk^1|R=kWew%d%MZL~KfC@R?4amj;T^PN*)QGU?H$;ch?r+q zf6!+s!XqA}F8cl`@=o-zd*N+$zjFEGOUs6gR$V`P6!|bsjensYon=w$+@|Thl&!gH z!lL-n0Yl={xT?NciqnsuRxWl_e^({!NKbyDrOXwGQ_pV5`1de>+WW&{FWw0mzx=j8 zqden)2RZhvDaX`4N}(%T(GHJrg-1h^D}|TAzl1i07i?CP^+D)b-pM#@%0Ak!&t3T7 zh|b1d{s>vyB&xbj+JdaiQ4PIx27Tv)x2~+R1y(Mx1uBQS!wcZA!Y{q~@tW-0POVa^ zTPQPJlJ5-7&v))<8aJoF?dZE>*|>%R_9^x-zk(Gn4Xd?3Ff8sYWnHcP^@os@Nz zGnMd)3cJ&n+^IqGuB@>;9pvqs*l7+%PQCt;Ulw)u!3%?f8;48%==ZSSo_au0y~E?; z*mH*-L+s=FkR!6^QQSi*Tv0LFxpM%0dfk=IX}S4xrLEI6cR<=Ib3YzmKiQ)bSr~Vu zpeg>ycbjBR9Ar;!uRX5*5OiBPkNRz=e#<%=^=mcj*VK7Vbp_={>F1jeZ^#r_3$D9$vjf346zQhu=?`we0s!fme$CWLX8e7~vv4oaEnK z2ETyUHk)JN@ha8#CnH~dO;b9! z9{0xD`8oVP>Q#dtT{Bt1^nutOQkn1F+0Sqs=4Uf> z0@0i6kn45qH7)0T(c44^_&MPq^3TCL_h+bKZ#KF|AATiq{x!70usx^h>qhaT$%(U7 zzebyd*Na}_!>^#O^YMJD`U&_{S$?2J_|#Kvd}_dYceoLGzpsxW|C`YfR<7x!Rj&Dt za|FEoF~*=WSs5sLLa5c|Z^16n!umz%awfdj(Fr*MuUxbY`9+#6^p8c!ozYLm>#}AC zX&2K*Y!Uv@~9;!%+~=5ycf?>Ix`fB#$V@X8JO0eQaze5E*s zu`_J?Yx7K-ekb|>qh`tk^!PXQeTr`N>e`NqE<@|`6sPdKN_0uDP6KD@ zGR)1Kb0b^E@e`RQwix}nx1%Q34;OdU1{0oZg?a8Ge5V!8jMMrPzC9v5gzznd zC13Ha(en{oKJin{csF|`db}%1%OHNT^?oC^e8S(42%kszIx9>*nS`$)j14j@<9H~6 zv7{^&9nh$=U5h8pSnD0iNG4oBSkfIUn|%X?_&nkzAN}3TMC}lLJN($YvD$vZ&WNzQ zGtdf?{xHukw!+1B?Ev8mtT1KlA)HHC%9c59^hdJROnkPLj&}cpFuG=n{+#fC2=}zY zN>}YQ!l@DAe-ci%!t7=Iop5KuV#AU#FJkUk_aoscaxeP>{~-Rm(53ZkC-PbDKQz-{ zZC+nB<0Y?uN8F2?m-IR0wF~;R@~P@*+@EijCGVYY<~5eC?=sCqgHX z=VF%>Js_I=ck`@(Ip{CMofYdX?UCIDc{Gg}34x@zO^1W9+kqFRP7z|FBEr z-=9btU-h9Hzv!bQiK}iay{#d`)?>QZJ6`UfZ2ds(v|}SSjl9KK7v|Mnbm4cDZ=HLg zT^TI<<&Ue5s>gY+eco>ToA=P}kiGK@@s#iA9ly#qle(jW*5HHUD?!J^hez~m(Xq3_ zt%be~d4UI?%?s>DzmitybSfehI2~g7ocZH9_yw z$bO#=z0SZlElfK6ACAa)#4{eF8Hdp_4oC8v5=y5Hspa`W;rZk%bw|foo5$IbX7W8N z)>i!t{K7%M%DBtkq1dhDUTj2i6&sLT3wR%$Nw?ohoj;~c<*XBhR+|}{Fnd3Hv!N^G zemFnhDeH&gc}k)5|73g}Ynl?B<;;_ES^sZJ&=xc$YIzk(^@H5s&$@Uo_DhjXd$5fp zL(j{Z6D!~`E7>orwxG@d-5Fdzb!_emaDWY9D>HjP;V# z$JaF0=^3dEs-{jp){~+GiZ6GKP1(6O8#`@3=+}jg#QWyBHV>*6S_U?U!ujJlS{Idb7qkcJcdXV}pqkcZopx``xluxE6Z@~@ zq^wFqR;AaUhhDNf7Q58d$~xb-L{{}YPF9`&ePk8$Dw?dS`u>4IGfq=hErO;*R#jq; z=q_^V`^X8#N92?bo6aHTu=tGy>1*NfEy;%cWHNq9Svdo=3ieV;YO%LrE87<1op3k& zUTkH;D{Jv%`5SXIWIyYL6y|G>-YKhuR?c4S@~=v8R6WSMY4BaK)926@M^bWCchiTB zePJz!{;RR~+?nZ63Y(Gp4~&E-QV-Uo(_P{}25(Dl#=n;OZa&(o)6+R8pru6gtkdHY zDLS8c$JTke&{!m&*v34}THNaxuejh4JQ&&KgKj*;D=d<2kjjx_zOH8Z()6GhCfINhcKe&_iEAve$v1?y zO2+SQd}oU6={pMstXOG-&b{mvi%m&AkPdKlRyzIQSQJm_F681sYB znve9AH!9kqUaD609QK6igZ%#fgZ=gPv;!5)+52XXcQ&uFX%CA%7k%Vk%-J!wVlzNi z2;YdLi_MIwL+zwFRP4*hK_9ZC64}wW_Z9vsWJy&GJbn!_1b!b}c;(~Xg*_f`!FF8< zU-Zp`XYIyUnQ}xPKXIUSx!8?G{?=ln7n%xQj4h&U(vW)6jJn!2CkOk~s4lLC9PCqK zPmZ*yh&@W|QoecEsyiv`?}W~ps9)K`iQ42mmoq1Zv#0RB8(HTUv)|sC^MT%jt(VHZ zqfyZ+HzjIfGkDalY?=O>i!^`#L4IWSmfS8Kw9#3rR^Ml^A6wMYT=gt%n#>K(m-wmI zXt~Fpw4=3?zlCvFM4Hi)J83PC#`{Iz+d*Hp^mE`>80T+h-rQWPRR5MSuE#$9D7NuB zY~y~~az1I_XTCj#y;Axhb1(blu{P(w%3<`XHyHvi?$;Yve2S7zO{gc)xo~t1c6)`MsIvPqWU89M8=;j3>H7 z$e|3<$929tyr=@XZjP(S@kNZQHIBOO&o~Ze94$SnmhsbhL$>&r=2P!+X1&MhW4H%e zji=5I>MZTp%lI9{*GcMMC;Ap;S~~0z*%x6gpvO;di{DSYw<1An#CNM}yEcgJIEQvv zdXD7lInuiR1Nz@rk{?(;H9xRhWNZ8W`AchI18K@zS}$#6JeAr+?L79v)w7kZGB?!^ zS_fXi^F`E8){_Zp^3Kb1T?dOf8zDNB-e1tZ*x8rW+!17&5TFtH` zXQf{aU&_3xeq0Sd^c$Y9=lSi-iLLnCJ;L5AbmM&(nYj#_DMOyaKSIngd@;hh94vB$ z8)X05p8RMD`zIUdhafyz^zh%v{e}A8^HRs&{<@t|iBTSR>~bG)e54)-kuGHCd=zwx zUcEF=vFoy5))X$@mp1P1jIJ7D51M(e@8_Ik9$p5W1s`$AUeMC4VpS78Uwn(C%=^g4m&Y2)o?In1>Nwg}$9!}!A6w`LfA&D}Z&>QcvT38JXQ)q-ziyLl ztBjwtrG+_Y?L%2@Si9b6gKk64=Dd_0U*j=myK8O8GU1&zz1{ziI@_sphS2I*lUBzX zv^v$rnFmL`1A3J+YB^#Hr`{gy74ywK4EFos<;nHepg)PfrP#mrBx3_ZKNkH!Y_3n? z-yqi&_**zxAM7E$tPy^TZ^B!54{5*-qBY>_fW9_iKWhXwy>}_drd7y8Nn@-JL~rO> z-w;-+45f`8FGKFNklw-#S^T@G$bI8F{)JTZ?sCNgjgbyqqn$H~K!)_rx-=c@ZJsJr;0Fy0f=xQ4ij zzqpU9#7o`L!Bd-GFnO>mQQL$a&*H&W{oQ+72Tr07+#Rv|3x9v9b^Tq0C10U~I<7L7 zlCNdkaA2SCFqVK`zVg45Bh3YA^k=(Ap= z>;8SiuJWrRrp*Jr_Cz+1^B+F7&7%|f=<+6r42ja#jUTrT{0sGnw5#j<+w_NNhOa+w zvsK^4{&tx4gqJ!>Kk8}3c0&C&^88kwN4~d_dAxym5AhGfPlY$AwCy(PGadiha~Th@ zPs$aB6LuBkvj z9grXJp@-?ZDz=&$&NOg_Ggb5jkvj><@aN$Tx}S%xdnVND>yLk;A4l;15OT+Z9wfSF zF=J}+*zZ!dC)JE+%&qtd#20T;G*gkq?ZK*To)cjI!0oUW4W9B4gCZdMA$a zdzIs<>rlflVbOqi-7mqx7+2vRBtBEJKhc-EFDjyaCA4!P>8sT6f+gsX%dn{}C*J|~ zfcRO}ypiZ!3cWJV*XDU;Z&v(O(&{&=%zySYg=S@*Jz~fVZ_+?P+nIE(+ ze~jlhz#F3Z7Nyb7Xud^Vi5K6Zx9~9%f3}_YVa#Pt70h=`95p{};^+*=#QJZi4JbVa z8GE%dM|@D@&rt?U>!LKw>cD!2Hu^>icFPU;&|Qx&-F5iXeHY)lYw;_b#9Hc_TkyHl zeG45f(q?y$#FJ);sG&({yVn*O{$)vcfZo(z4-Iq(~}Hp3ro;;Pfy zNB6J~eovE+v%6x`5n96@0WOL6m1QMynveQ?mvy)7n>Wutp8j4Z3~U~DI;FmaJS+Rl z7%)DAjj>$x#CX5>8Z0icYd_g!)5Kq`hH(}eknyZyEPujSE@rM&P8VBVg0^fX_y3&p zvrE-`pA=fHS(D(j{Dv5t^|J3$S;U;@Y_ShouY}fxj)KssPDjM~m`~ zt#1E^Z{!l^6WU2;PE1U6{u6oWV@)Nt;qLT_*c!T-eUhOnyQEJZwiO&~W}I5!Px$G* za-(_HPR4& zWRB2jjP{7+hh0KnTh4Er$HkIgnHfK`E&h7qy^Q4w-m~aW-+N-7ETs(xE0lmAol5wp zoHv#A#Ru3>4xvBX0Iv#>XQZE0Ep{F2yy;|gNjYy?hn>bcZ+bSg{rlI=^QLy=ylE}G zQ09p)yLNSQUbZnmFyJ-xAIi04#Wvc!o&LAZoO)ie&YbqB-@>^B))j;AB>hvAU&mFq zM=`#W8CyATD&v@WBjhWM-Vyddl_16GGvYLF*%PbHtCg{>;d0) zeAV7yPxS@n;9JCPfZld@a?09T>h9y-m)zNDl;<$*&pqmw{!6W1k1a_2l2=oo9QN;o zzE%9|c0wOdVLPcSnKVc2FS%K+hD))3)L}!>{Q#FEZ(Yjz7as&k{77V z#)d_|Rx#(R;K8!*zvx|5<8S&PJ03c^ ze)}HQu_v&ZMcO?rU%Y61?;`0Z^w~NYn^<^^$tU|td}lL0LT@=w;(MC$8KgIH-p-YH z^d3DQXy4Lv(ufy+`gX3tPlwDV-4ZihRa?4Fdb$|x4fcj~KKo27w!OCJ9eEyNt??9N zRt9d}>^mR4!cdgiKAH`N%N?y5HT{PW3 zi>+4dv%)XLRx5VeX*@rfo%S#oEoID2_Xn+)){<87CTQ90&0Fiw$%k48PUpGE5jlTi zts6JKqgG!c@!1`Y#mkwd4a8qe{H6O_Q-#M|1q~JO8_DlVepm35_|fK7@23}9*B6^% zS>t9ZnXl@)e(oo6RGvc~2#$)#{2kcB3&2b{zqc#gFCD%P9*SV5WN=+TIyqa2{b{Mw ze!g}$ZM)u{pvgL?4*Zqmxx*TK*r4!B)A{?U^0VO+bH3XZ{z;iz$6#5&+W7o)j50U( z;=TjVk{neVz+dT%UiLWnD<9Z9*SD$${>obLSMYDqZsY9EcT^=D4~;G>oz{@sh50tu z^ZWZ}y2HOWrOshs&Ja!LF=<`!GB>FlaxQ#R>!M>Yv>?skprr+3paTz;SDVt)zy z*nw8|mwMFy6k7Zv;}HjbOAG6tQu;*XNG1A_%*D#i%6cz-QpK7=^p(XW#QR731PmXmJ=s=u7c;)@T*92ivG0- z-949iDY6ZJKpodb;(cr2Q)@UY@ib{R@a`%*{K?$kWsm&DUHB|uTb1)Y!k;|kZOIuQ z_hNe#+1X5<_h%<*Uh)z>Y_aHo~3F zzLRo3tOhyldzyDQz|&r)PECf+S>D)c=ppb0;g6QAUM;>v$YPymP@csz62RRMot8dw z${KYA^x6V{UdEdB&Q5VvliBkt?kE@%>Gk)HWDfRHwE4DfI);SU+9mx>v~TxF?6gm~ z=eA^z)8)V5JX!L;eVHHp(6H(GD6_?$+;@k)Ti;PxarKwFQ}pqEUic&BXPNwQo0(qp z%@ttD$eK_=FLQ8qR?eSfp%XazL}33h297ZX-&4l7@wF}FubC9VS?D09QJw9|?V~;Pb{?*YDdwjeB4RigrJIaRuv8Ur)x2Hi|wW_$8^rhletj$iYQR@&dBHd;l7M5ZYS?O|F2h?}x4{ zq1!5GR@Tsq@>y4M&ZkCnl++~Wn}0gGz6qY~U@egQp@G$r^C9O=8K3r89Zm2qp-W4K z)^YE{w&pup&{TQ!s-{JP&4XODWSQ8jMXr0EM^4ZdtKWV|zX`7_HgPia^@E9%al=ut z`*>#A=oZi?VxzkN9FnJ*=VI3y%l$g;RiPjH`k4OqWb3-6v_;}}P%rEG{bt<1BjR+s z@+y&EZS|T&z1}2EYlRAi;U~}X=Ul}8Blbz_9Oz*DCI&fNfqk*;=^@)f*^04FNLI9e zO=qtap75Gr>U3tkWuB*K;XFk!Vx5qM?jh@hP{cZ+t75A@@ONXKP-mVKUQIs;4;+oH zM|9pa#_Peq<0p(Sr_c)uEQO)^j;Sx12Omr{2`*Amfow{;d_tRtpc9GsyKp?3ZrN zPGBzYJ%0s7a%rE8>Y3-7f( zPVsBL<8g}rH4mS%&wx|>I(#)6PVpAvBXNqC-**B`PuGTQ;#1}tIq^p>kr6V*7p`(X*9gn zYoHy$t%$^a6@KDym~~~IS~767-u^bc)-JUFbnse-ou~}xU1T0|^BAv`^VQ-jqsK>J zXZMqM_Ko%U2)x$b63-ZgBp=4is;kxJb12)omv(-Yms_Og!^m54+CR zY6)BS_0~P<9^qbOg4pLouE;wU-?uO-Hp0hbzxq+ivh)1;!-j0oWx>PfT=K?ITwkk@HYEIC&N_G=R;=tF-(Or>5G$KDs;p~ z={0>34s)J;54H?^EY=kWro!z*+WZlsU@9283uU3pSeOb^q`d!+FcmhiXB~;DpwH9B z9@$t5vbQOA!qdT2=mkH}=P&pQ z8QfdEpaosl+HY;P^uUkJxldY|`^lsal5QdG5ln?=DgWz;sW7qhvtcR}Lc1%_Q!Pw| zQH1}SF%{xze>A&fjKs6%5#7WaVV6A0K20>c2^o?(~lil1;a zyJSdD7o+_KKc{H>KS9aA%*=mgn@;q1i7zqZi`wE}mHe_fUqOCz8E;wBcH@l9r^Qsz z`J}g}u^)o1vf!Vl9~Ju^lIPbQQ^DZ#J#{~E@$XHe{wKp!Sb#jqi@;P!JE;DDi>Z(c z&;2Se753r}wi4UgmyM~=m377G`i^{FOod;dFPy5ZPlKuOBKdt&Fclh3qwN-UMP;Oq z&caWJsc;rLUL>YMydM4}rb04U#a|Vs!atyO!BnW#&(ohCrUE+tT1R=`&xEP)JpBGt zmivK=>NMk$#ZU237V)bvjSe){{ms6|6G;KsXXpL60v! zM|+fb3sXUl^JZzkB5wGxchA-yCM=i=MqX#@?+moUq<@6xf~lZCZ%otH5Ee`YtBe}L zk(dg4yen5*OuS$!=waIJBh0y!V_}VOPb*y9NAnVvbH+x#eY8r#$yS*B?j!t7$5d#d z4Pq-4-;hs(sqiQBJ=w<)Ooco!b|sH50#l)$vczV8F7r(=70%`=m+|); zRABwFHY<3p=}(qcfB5fh=b5wMmlmeNc-Ctt!&Lb74^E7!a2c`o&=?=(4r8n~tfl0qpj#5>w%Q*663gRM34@J}0Ka^!HAVso=o3Q~0T1F?23Mau-xPkUtZ7=;Vz*JD!w-)~(@iE`W+1O~93NQ0bn6ERY0=^AP zvm8UU66mBv>8;z!wsC%|_iLw*sW69k)AD!(j*uYA2Q6>ggp z9aABQ&*|x4D$H?5_*;z@f2%Wtsj!dpb$i*XYmcdb-}2WQQvn~{6Jjdx4T`9k3RTBr zDtwfy+Kyo=l#*ZWUd}>K$FEH=6(ZXp`x-g4LumIPdoJP+_h~T|Y8g+#RCt;FSj(UK%f?jb zQi43c7JrwR6JsjeOg{uoOoi{sJ(vmw@NLIygHMX7fL^^cuQ*MY{p~Ro{`dxa(DdD@ z&c$1p3c4QAhz|+-K#%b5nZQ)=wZl}v_PNwio_gAt3OD@HXhVBU1-;!$+Hsf)od1E( zett{^g*Du%Fcs$Fr}23)6-ru8j;Ww9Cn7NwYS`b9IdL*fg;#HH!&LaI+@EXG(dt2M zJ`7PX74-hhOg)Z{E~P(DMLS<6rowXC6OD&x#7n;jAGwY`%Hdx8sUy$TT9^v!puZDg zD)i=DO?ym*MJA@gqwO#ibXnp)?{mqLd-%mM z(z>hJpR4ZUsr%_-D(E~IOodyq<5@h|;^}7!Q{nepj>l9u_(~h5LL%#-XqXBc$DVmi z1--3cDh!Cy)@Yat`n>ZR{xHq(|5<#yq&=pBoY{;AFC(Hi*SuqvvWWO;IQ{lbXw)OgC(_Y$qnwSc~%iw8pjv?ZE zwI{+<(B*S&#xYEVX4Y?~gQ+l0pC_I750m1S7P^dfmO6v-qMa>HCtffW{=s|VYxhyQ z-265lxdh2Ciceh!;ssM-E$>BQDlBH63}t>r!&F$o{>*2^RQL<|o@q>ldC( zgqK=pUBw1`3G@<)sj!r~2&TfXp?g_#M8j10hwS@(12Gjcq1i7AQ=vC~dooOgbL2TR z{AFS)%th9H-7pp2x`lqQn|ux3-#VN6n#4!(U4B{OqhKmLFY!@)mp2hF{B&Wu!B4k? z2^`ILIiRPD(H02ig7DdPm~+H6}+SsOof-Q>sag06JaXMcsV+z!p+2goiG)!Kk1kX*VDG^ zOESNamV!O`0b3-$5hB2d}c5eR#2ucUr!TL;U{ML z&x@%rk9?wGD%?qYdrXCQp~us~RG80t{}^uiXT(%^?k?!y8;GgUhkg@Gh0GY?$LX4` z-$v zll({67uhSg%wr8~vxzH|>ROvJXdbu<9{X7Rn{SK3>!|CY3@o*SjnFU8KOK2IBQ}xm za`&t+{;B;yX|@tBj%^%XT=An}o<}?QXFSo-e|L7QX5$+p^Rr`}at2QHPWf~TUu`NRBJ zz24*azM!PbCbO5pS!MB3iq()+YJW!4|gYMp54axTi+w! zP@ZzBi~Sb`pWS89jbN@y{+^v+Sn%D+QgFQl=f+FgyS9HHZ2ga3(J|fVGY|W4`g!z; zZR^Bx_^M~58K2)4Kacq0e4BPT{&!BswZGYb630`Mb8-qKoO*RdizZzcHWOjW|O*^~1OaD`Kn7$fW%uQH$Sg0}Y9 zZH$`|Ze~3A77O2)O09pJy~EktmGCs)DeZryKc~0LKV$G%|E(1^ZRU1W^Yl5#f7gI> z{kM$B@ZV}z!_#8b@bm<(^!*O{etw0+38umwIeZh=(^(B~zLm0*mGz!Se&82OM^8^R zT$-VVe@h>Fwz~tf_JgOyczUvx!OYVwietFU(=B zwOhw*I6nWgz_w&v#9r<|4>&NT;HwCpwT!z8hSP3r4xXkl&JfSrMA+Fr9scWw-dC#!Sm_AIkgmU(PJH_Ha`6q{MY`N$>H25~3HP?bJl{t+i?E#C zI0*fW1D9_&KMDVUw#!xUh~;YG3(I{sGfwa^1lL%2g9o~sOg-eelp$B$J_6>Sq!IeG znfR)bR_p-RkUoYw=-4XI&F$v%0_a)3JtKJ>Hm{x{b1sy$&4mW+4KjCf`Cp0d$;qL9QKA!{YGmd@~tQdXVb(>V0eh%eg z@_dy%YAD0_Ua)bVrJ8Up^_ZQihKr%!`Oxojvmc~?T39OyKGfU1EAu{hs8z=*EhVjo zcS@mI3#&AjHIJv2xs_^Qm3p#yKE@W9KOoK-WFAh2FYQs`soBOi=3MaiZALD{$(F1LyZ%nr8vC|i;g_?} zFOc@H{GR8xnV(>CTkZ1+Eg&-_FB#`DWNS`D`(Fi5ss`MyM!rYni@=lewZoJ0jWNdC zJHBnahtbEr{$~6D;!QlMLB!Vx-d6{ua5Gp+o+dTmnHa03sqhTOy>toRZLZ*Zll1$j ziz`=7 zDrHCG-(4g=3jay~k(7~Rmf?FDdXAv;qnx{*O`2T#c{snz z`03*q1qUw=oG!t^o5nbYz;uoS2XAa|@a@wLOgh2A8vzcU4Qw=#9g`U6*Kf?!^l=CO zE^V%}f%6j@^6Zp(Y(uV&1^e!n&Ib0~jS@Do?|kGxx`Qh`HYu*E8tl6Z!MIxt_TBsT zPWA7Y*mq07zT=xi+DtGOk{GKT#`C+So`%`fUF3`-apZw8_&g)Pt;=G*$T_ee{U&Gr zr>}8nGRF#jIZ~GyGoCK;j|P*@(~r5*Uk%R~fxO9A!?%n?_Ke{kxxNFrK7VSwF4yM| z0Mm>1PXjk>{!o4;@lJi7^-=ZjaHN5YCzyDHz{Hb1dkF1#ed3XCJZ!JYZyd%GB`P_~YL_o4DQZ48BYG^rrNNQFiD78W%j?)yvea z@z>!qoXmHY(5Xak8pVAsV=*dO3A@xVvEgh zRd^_G_MyDMoHBRVL78n>yy;-^_R!^wg~#jIZsGB!*1yMD7qK0u@lHDB%eyk}B1d&z zq}z_u>W7VG{X<gd^>;570*2j3zCpXSMz)aw24q*i!|$V2PB)y%2Qe3wk}ZA4#-lRk$hdq%k7D{-1< zBJ_1FW79-`PXtF*cyvX)CVJw&_gk0muW&hA;du&rhU7h&tLTxl?eSVM@5^`Lrc8xj zkYDLw@?;$)*o4zZy2CT_E zU|`&B{y+)nyrIw_y#-5RG@CzmW3L_rA9<2?}(guCZXhX=J(znH)+BYZ5QQzI&z5YVlt;=WZ@iJbw zF(2Q@<|q5OFTJ8xe~;&a@hg@joHK!)*X_HX9Xl@{z2-#N zdGn~}r^U`=T-N3>E(wf@=y&L(YZY`((Rtm>>Cwbb;-}KzdS70mIJXiO`Rh3ecHU;v zi0(LvItg~(EWPhOiJiwer?pu>8m?t?Eom68T}k~aqhRN4(93Vb&RamgoGy0WW0Gf-vVJ9bMqualhsOR1PWqX^&Ras>r^3$r z@zfKHZ4~Uhd3yWXu=Db0|LI`o-EyKbqG0FUDE$-#JMViE9|b#alEj|`JMTM`{dK|4 zdy(@H`;bB1@hRG$fqs$?9y2~S;q!OnY|vIINtAm0iQ z*`UjcCy&R@%V$jVc(C(aA`kLAY&|@)Gxn~tuzPjE{?!#bSTgpo6#o%>2kf3|z2ud1 z7I?C+f#Lg=S{NtXY)O}yDn1#^TF*ph@E0ze{`UH zSGbjL4z#f5yf)kJuPd=_Ew!^hIQcTIpihjyF5kAbpic*X0psX!_dh9i-d|QA)7oL@ z>GHF&|8dxPdtbo5@&#e%C0&b-l6~f|^S1sYI(8m({gGhj-O8Mc#Li11d@}4jeLi?E zJcgb33+aoKVCN0ShGJpoUBLYw))RuAmm}DDd}~PIJKKul9j-*d&NFlu%BpC?&U--0 z`~L_#@4f#$9y?E;rz;J)axZ$k(O1KdVds_d%|n^n zp9VW`BK$y~zu@6r#l58iicRTM*m;+eK1jL-+9TL`A5s3-5j*ck*L*hYy!)nDzJaOw zdC$8E|2Jdj4W<1)=CXcfi*dJXD+48-b-o^tEyG#^_m_C=8+ttQ9!pPC9&52&iSJ{^ z_iIa^O}zM@ynB}6f0B=nT{QobbUhv4z7W5ld1gNI+w$om`InjTGuz@5B)-IqFKUbL zAo*q6wU@|mF7^AY*m*ji3|?gTyu#VCvbHn z?r1D)mrIp_@*Nxbo{jimc^86nQN?!%_;!tq&4ML-+ufn0EZ@yHo_3GZ^@v3F4?}~G zf3LEJa_8j-N1wyuX9@m;KTm-rhk;i7dm)8=WLY^igAf!Ps}$hK=ZYc(gN89U;GO3U*%0X|&xsi%}VQ7GvS3!_IT# z;~a^dH&hRQ5`ajgFm%-n&%s4n1bGJO|m5%TLyf=05vU#dWck zN}OQl>FG0vYsrKKJ5LYu{A|K^TH)XTErGCL=jqQ0rx1?B&eP*-FV$4yBeC=Jcx8xo zi18HcJU#3kpzS9t*m+i1;XT35v+_C2^J@r4!_L#wWL~KKhctqnr-v!yL&A~Rd3t=~ zCEDA>TiAJeTyT)~262L&XS{of){kg3cAg%lj4gyC zvGer!+HY&m5HHwydYE^#mSY`Ok( zV&^^d=E<@1ocLyb7VNya`1<}IcHUQmo%iB*PAn(R6n5T!659Ch>0sv>JZP}VgHr1g z|J=rdJ}q|MBv~t*1Uqjr?RT*D5Zu832JF0x*|)Z^^ZIZf4LeUkFZnuS=OKrdX4ToW zY;0d8#h2-}GQrN91iu$snVbWu?W`334f}k^e%1>q_(=EYowCaJ7B;KZF8`_oN7WGC zO^5Zz^`)>b&UKd0pW1r-Pm6zaZjV%^Aeb>w|8wm%Y07*m)kt)H&gv$<8&V1*xjAv z6q{@L_C)Q81Fg#iXGy;OTFZCsg{FcRD}{AslZMobUG{3%oE*2K@2D=Wh8+C5*75!M z$nSFRl?d(-_BW;o%mvV4@P|7dQ+mdh`LYb zJmz%1aqaIvh%*z)mfWo)j$!BJl3(t|%T9_J)R#1)vz#Zy&dUSO>g$S~7df6^H|)H> z$-W4_mioR*1a@8%@q&5v5dKSoofp{#+1L2I*m=c_r(owP_(NFdX1{Fgyu9(qbIu)v zm}4P)Joj>bYA<^^4fw`n*pnYE!Jl9`{a|6|{Y36B~B$*a?*w<#ER@_W{R8 z>VXivJ7iDthsu<2FWQ&ac!4ha+hgbL-NYU=^Zr!$XBKu|7I|f5p08u)y~(?00z1#c zd~`4$Tb9{0fA&Cs4rMr+&p&PKyv3T)hW6Nbdb^7+K$h9@hqY-gXeA(ZJ_Fc!7o$I& z3OnyQ{4_o^!|cz2_fCM^os}Q_;?siJiBZ_C(_$&k-;EB79^KeU!ufiLmo#L4PO0&YQ#;x%Sw3 z)2#FTipZ5Nn!n#5|Azj9{kkk!?fhJ_BoocVz+OLJLt2)&byF!!Oq){kBzKjvhtNeFJt6&>9TaIkFz^I{6i}74bji|u-=KoN3e1{Yo(#? z@B+@|EE*86c{z*Y;2dQY{z0oY#cPW=kJFd7FXBv24QFz^3rSz4;v1rdA6lk{tCo}R z0DItpH{g|TBs!NuuX*~Or0mV6bDlJ<{^$Sr^w@cWq4{XOMSE_5_M-R}eL%eU7ClY- zzbx#$AAoIrD(t-B(3fE64dyDCZZ59r*rGlScHT3zEt3A8B>YXl&btVI+tb0$`xxJh z)4|R|er(Ar9td6asCPh9f}M94eG!SBH=Xdwu=5NVH1rs4?tA<5Vdq^%dC{=*MiU>2 zotGF3eS)3mDHJ+AF?QZC$uEje-5}zlVdp)=JefgXM8nSeD>jtRik)|We9ts?-ea6g zkn@jsVhec+-}5@I^YQiV|!W5&e7&ifZ~{L8@3d;EIy3?AnU<$Pi^?7WTCMX>YUf$n9^5e+-96Fl^rh@E#e zH2Xzi=S`w-PllcM9iC$c+G5!z3zD!+F3Ag2W+PYoxWiS<`6`jMoO4|yviL1qpr(v- zwD_-?itn-_@lkx24{~{+Sn%z!^H`rQ9gS_MYP&;US7WbTEB0E!&O;B~lG{A=gxGo5 zX}6rrPWvYIVGBEtHQ5%y&TGZ4W34|=gq`>E4bidlmJ|PV!p?JI$8g%u*Dj-N*KZp5 zjl|BoW|$?LK0S8cG-&a^4Li>xIv_sbm8;-`_=(Gz|Efpb;l(9MTIJL5LiS~jy5 zZ&JgH{{mlpfnRcGE!0m5*Ccm#`kv<94eqe-Wq4_GlQM? zSIX4o>uF-={lQHCd9m{zBcEv4dA}lFmmgr?S@I(d|DMyq&RhGfh&{f~h@H2a@7H}3 zvGcx5zX^68dl~TKbj?_UCEVu2o73uy;rdEdVw$35v;rgVCB_=mG|q8{&~Gj zth`>aPC553m|`2#l|qU4^cv}$waviCO9V&6lMX)KufWHX^U@UtK3*K_mrzljKa8xm z@bNqy6&)XMFXh(#)4<0IfRFe8sC)bHsH$u6|D2hS%p@edWM&jdb=#Gz`6B%te-&{T`xz%BSzlO%@}z@ zh&MCbkog-%DV6^ToI?9p^xSYTT&ed_*2L3CIlIAF6Mr^m(A+KU+gI9h?@`8^@I&By zWs;t^DbCD0S&y+dguFH6ZHi-`^vLgd7ad}*(~99|!8btJ_7}!a?UH9Sv24!e3|amW zy5%t^DzP4GcF!nw5MyuYeWjtrhss)Bk{Ek}@2V2*a`0_EU(K_uvzPI&p5~sYO(sp^ z=vnVq@E#dQPfusv{CLtOj-K^i!h2*KJw2Uu=0&8BxAO5mj`zqoddB~kYB{7!96jT| z60MMTiE(FrAHjPb@3L=1e9$+@o-;0aUqJihE^+kaZpG1)=X5hoWJY%!J^I2ElqcV% z4!KKAsQ0_$=%x9>JXc76BCqgTJ^9B$A3cs9ymec5oIN9so|GYR^yFT1li642_Vm?{ z@I6T1*5l|!^ld-s5=U<+WeW@?5=Rey|C;%4J&sSxY+al|ygYb$8N4j<0CfJ;{aE53&GcPMJbQ^XC$LMKI~g~L zT_bVyYT!XBo8uG`^oM2HJ&`ISKwd_WFf-3^WIGE!j&hez`(oN|n=pPg6#B z478M!W1s~hVxT?!4Pv0(Kn%3S)GM@=7-&PviCxzn11;0Ho^@+_4782JKm%{xG0v{*X-VrVtT;*okg4c4D24*s`{$*ol-Yuu0kLaw(Vh$a$R_+A@E*F=tRq ze7X{83ow5l_#~znezfQFBIbA2GUhz?@#WrF?aDa$dMMImtT(=mgzfJ;_gB3_P>X>en)6I3;KSC zxO)rWvD*w?D{=Qkk7Y>=IMNq?X=|x@ow$2$9%wk;1#`O#oa6X*owQCZ`Gvye&O0k^xs*DNz`UR@&gRGRM?ar1)puHqbge!9(vf3FnWFSZdgkAG$E z?8o@Gq^$W9Deq!4X5JOV%qt^io>}(78==Qd$ilhQdowZfu=kc7N-ACseG7#jz*`OP zMq9_5u>iAtzx*{iSmKjXui#GFQcK)Et35KFt{G>_E$X-I-c)!}#?$kWu_7D9PVy3S z?_OlZ%Z#TtFrIGI$J3dxRIUMKO%3I{oMDy7OUo9do*NhPtMZh3{c zPAYMoY{Yf46W1w@xK65;MmZB0H_LjE^!~cu+IpYO6*?wyV}Iy6t~BJH>ma6mcANTC z7He!I79_DlcV(;DE4IZeyY9sX2r0_8$h>=P_Fa;HNjSW<(BAsKV^Hh+fdLge9XDLG z&RIJr=p^Q|qRt8C9eHR^9sHIG-*thbLdq|SbMLMl=~!1w|E~d#PMiCwhJ5EbrP^rA ziKUkTe>v?SHvXJa;#K{w$xi!Xr3)?`w6B7)1Sg5qH+`(4O+WbHQw}xJldC4~RL@p4 z!O@}FMm)LTob8mYQily3fhX2L2#)^Dnvm1sD2p}=ejMN@OU=^p^ECJgo`j#cU6Pj= zq+1;(ejLE&NM8C>yUnrFS$k8k^F(hPy@lMCb&w4iX&&{zlowo#4+KAs^t7Eyt%aL% z;4TCHQgHLzrdMgxiKV^M1Sg4%ImWPe#WBL=tx8r?lHijo!H?|m9IH52*yjJ>c=^iQ zb8mz0Z?~z3+|>0awe|hapyMx-ul9VVI#=8=Xu!^a)W2RG*p-Wn`BOh-*T2-118u~p zE?b#*Zdud$=U%0Z9G()VOq!lY{9N!ItL)Mf;Yx{He!xCpXX2nlPuHieAKjX#>{?D8 zPSpvGTqguBg);jCk1~JZad}SWc_r{P16Q-$Pn7R@F8NB&`m>aSUiv}*f$munPf4Y% z(&qle1zd^E(BGkiKFM*<>OX(XMd#!x#SVDh0ql!->3V#o0Z_cYm+ zZ4SnJ^PhTnchR(89a=u(%|p|k`tXqR!8g}AROimZeia_aiLyO9Oti}>^8D=fwdug# z^z}n~niwNC9erp|PM-1td|l}{OVKjn=NHk5VOz$t8)2VEfGnGt_9uEP0?L!_U zBhws|A?3;c`zYsm$`x671fQV9V^}mx@!tE8O5Zt|?@laz4Vj~0hbX=(i4zbJE5hM& z5&vA7cw%W=lh9!1Hsl?A;ZU$=CL42@D*bQ+V;%)s<^w}N_`3smBtQ8=f-8qQKZxu; zSWUZME4IHo@{-tbA{7_@#hL1=du*k_Wi~Zf|OXR{Qv#_)4dSXY8 zV_Zi}T|E|_AD*!C?IWM)Yr#cTnW7z-Vf;6eag&sH3-UjMb6(TA4qyEn@B@y@72@hC zK7r*p@f4zAIXJa9EN_~y3_lAS&Q=z5K)b38+Wp^)_v9h(1zsy=2A{J3hYt1d~cZGhCVHAVLToSH}Gh$*qTQB ziQ^~Z>8O8hHQRqIif&HMjuD>apGh^0SskTLS?3nY3Aqk!0QM?iuxRnnR5k6TUFxbT zVzw8;w>9}nvBX_#rM>|4_tSSe;3eVde*-q*=cCuXiY%9YgdV~tjUIbS_*(3Y6HCkZ zR*Ib8q4yQxp?kib6TBDLo2a+ONN5jh8p5 zk+EwYap>j0e8xla-{`3SM(XeBz8AntYJ^7_x z0$JnBXIvC-;4Py{(JsOdCw{rFz<&*O3(Xz<93JrlNM60q^DVkep^4x{%7Si{f0-*~ zXZS`?-ciO?V&nBy-rpE6iY<7v^#7zdHp^ek3GC^ z;ZkC;S~!*V#?mRfOnmG>{z;tGM(`>BRM~9WkQ4ZOU$<#la-XYeK~w&JEVAukmoBsE z7rk^wTt2 zcU%4BtA9JW&y<;To@Cz7G4FZiU2G+xsmwF<@I&9YtmuZlZ(L64&eJz8CwAxQ8<%6d z^Yo3&5#4$E#^q4*2re7oW0^;YloyuVd5$?bksFglZV8W*f6Fu@Z5(MG#FY1QMeE`G zdrdt|TrTvTrqqUmF8DBEykQHpPCwb+*>-tvd*?Cgt{HBO-)d}^_2TmlW}lH5MQ+kvw-bQg41w4D_{qwD%6`lvu8MTd`T5{yMxyaI;z3DD5ZhZrU$ni;ZTQ zel`Q&bQuXQtM(~cKkOQ*&&|8w?_*@d>L@&k902!QEj|?)q|0IK9t&Tuj5F|cPXxYJ zF@BaZ=SShoDrXA(9*dU6CM|z!$t&9OJ=!N@*4rc6cZo^gUz2Zs6wHEyYk(zMUtOWY zaZ>#pruEbn3G)yWhM$mL0StZlF9cs{e=qZ54{E`9MaxPsb-iNKe0mv% zu73$UckEkY=y=A4N736=Wx@|9`xC-n8Ru6>S)#-KYP0FOel_JC#UCcRKC-O+W?2Wn z5?eOicPlUt1?C}KvGnXxpV2oWWwOPqf}8R1^SzI|f)`@rG`_BA@k@*`Rt5IGLvgHN z>~~P|H`z*cf3?_om5O8HYy0r+iTx)!IASc+VfF(lJJNP6S}$_(WF2V9J}IjS|3ze3 zGS4FVNXp7yudEWA&5vy>d?EQG^G3Gm>;IJgr|WRNKNNtY-TW7gpLF`fwX|F4DsBJ$ z;{ThrA3)c}YP+;S?1sMD@poOWrun{rwi{!>FL+4(v&*}UGVUugY?i%siiWRDw^`av zo28X{ESqIE{3A9?gITxOD}vu>n`PS7C&S!kj6a@?9Dh_pZv{M7U8bgm>wA2T{sblT zH0Sl`V-DseL|*V6zk)F?apWpY`MW~1_-`L$66_VPjB#K1N$+xqukVwxREepf#Eb9p z$$96*8@|g{rQ3Ho7k+&P`+WAA-o8uuAAekNBL7XV9juitnIB{=>yy(vQnXT@rOxqd z)U@NbpX`eZ?{MrA-Uy6RwA{EWnw2|OmZ*0=R+1YxmFLHKeuC%Acy8c%InS5#yn<&B z&t=X2M{Uiyapm0Y++Exi?xoz-=A-3tT3Qw|89Cy}QbICTQ<5E5wc6a1KS)$U;Z1SH ztvBPppzeLS?pZtFfdkC7DI*Mjh@a=9h7Aw{nz9XT6Q3>_aPgeLWZ>3lo0mo zR)_Ra1^+SQy6(Qhf1!zn&Y-Qxryf7b@!Nl^kM|Vts#10}asG4hSqnb={P%lZ{-pY@ zq|UGS7PhOqqJ5cqUT9Y#d(cbSo7MJOu+XV&)8DHTHHCM#Q`!1fIr3>Hhqu3z{mGCzOcuWukyUDCa$=BP|D8F7;7)H z4bXj$oQJmSk7{x+-{VYtk7MJNNe+CEZq?zDcq-9lTfWDDJ!L2Ua*z0?M+1M6Iy(4f zeDI}I-mi^QCXS$8F502E5;fcN=p(_MiL2J5;cfc2szhxY-yVp8x2mJz%9n_>Zs9Hf z?sAWYFWf?(e}Qzt9Xv5n#vbX4(t|(BmTynf$1PmTSV78BW-`Z0U4~5W;Sp%ApHaGw za$C)QJ&pW@wTZIv(IIF{tho)f3snU%hNNCUW9KTx^zr|yO!>+Ss(uBv}zNWk7-TXF|_ zl5j9ves&zjJxgH)LL}T9M((38-5WLz4SCbkkvA#hhJ;Z_o9nM zH@1;R|JwSx?gL5p9V6zB*iV)ZOPA-ZspQW_R%QBDq2G$^nI1XAR`Ml&zIoPY=CRiR@6lU^ zcv(jn3csSZ2+jA^$Mc`W-Olt~g8T}Um$!(|ry3tlfHRc?`xA>D3Uob~dNQrsY$mOT zKtvUlSqvXU)8{hUP?a#Vxhf&PdG3I#n&(~uzNePA zxRCc{PuQC~lH!`n2e>EubNGK$Mv=tK@YANd(7mf4a{cJFc=$VW{g7kNs-dI5R72Ci zO<+rMG3Q*igbTRp)lk^Oe7NBE5_?NvhP|cpCBAP;F1Gmm@2sH>6qL0HEvoTtRn`BY zq`Ll(B~_0mHS29}8?1H0$DK>z^JONV*P3mo{7&}5hS>uf*a5FoSLYKv(=UWqJc5H; zN<*C=$7{I<|5V~Cf1-r`AB@I>^o)@})1TW=pmjtdWYpE{(W69j|Ka|W( zP?`g&wwCJpxaPS9?#Zr4F9&bVX4fRv=*|W|&$9l4eY&nk%SlgYc0GC-cL(<}9l!qi z1Z^E{mHB3QUadc8c*F3y&@8YJxM^oN43F$lLi#!ZcPi_F`xC3AzfRLTIFm5QvtIY3 zgN?e^3oO+6=s=^+M+X^oK02VeVWXY%0nzJD_vESq&LOFPyd=Qd`%YraA7Jlw7ch5y zp@!7@CrY|f%UZgI#R zdD2n;I^UFLT}MRBcYH}1;y<1sI(DO~1^OuyrzN<&x%ezy$X|E7OUHF*fg0);JpibM*NWtcEjh`Z6hHQK4%?#V2jx5razLkGWxg#{(FFLvL;o^LC!s; zulvEqQaN`>&jWAXO&-zfk|$a}+%EKw&<_iF7oUIy=ckOx(bH)gX}T=T|VbXQIAo`Ye3op9M;7#;g;79O9IQ+Q-+q)CHv2y_Z zHC2^)(9a!hafU1&O8>~Ae~jVVJajoU?1T@S3e;Wx9S-f5W%zzLyD9I-tQY6qU!Z84 z_*TZe!j1XRW=DlSZ&$@wMDib_tsNd>ozSoR?5`8QeL8r(6nwbjiK`Z$!Znp^0M}() z{kbmZO2#%X?7k3Bu(&h^Geu*9Hj(0YTEgQg=HO}Y7=6wCNpdVW{vy@og zv~3%IQ>%ZxT z`ajsX)0)*)zqH=5Ys2PyJh9(Xu}{Trwe#!>8RJIm%Xh@Se1r58Z;^hYclt*WWpywX znhiYSKl9IXXk);a^!YcjR|Plk^Zo|UJGp)0e5p5c)Sn(!X9Q?wqZ9Yx>2Y{U)(H zrLP&ZZ^(uAn~b?>oyYlpDR6Fq_CLyJOl^&sSpVkc`9WYfv9to%ry^6Y@X{9DPPcIW z$#eKaBJi}a*?9MYPr-RCI0IjXFVx%0_{wdLuZA->0pI%e6&Q10j~t`laej(wjyDUV z$D6?YJ$$l}ywMTC8&?Cj$dSS?6DA&J&0Htz3#ENEin6scm@x58V0OX>R(l(R;mQYq zS!jAc_uQa4=V?=le?#7TB`u0S{u$A3;g6ljK#`r7LGyAhsbA_7S+^b?ajBUeuD9(9 z&WQ(~3E&leSn6a9BlfP?*EXLEJ`-81zylJS^qw+BtFo2;)y;aR!L&;sn<5k1O1XBE&+ zUCt;;hMZ~WDYq=nJ?ml0{ggG2(Y&^aa^?S_{4a7O1AY-*dKUQ2gkNH{!2!SAJGitZ z5+0$o!1FHf1n>#P^m`R3+Mm&RGM+lIbQ-v>0OwP&3*h5=gO9^+_Q->pxk|B|Z~rA} z7Of>s~@-TTi1LpJ{v8%X@5e%5{*8Ks1_Ae$SJ%`Rl~v$XBsvDcQCv)-Mx^}6p7eT~gD zt0_V@Z>`V1$or>@ypCA1nX$cphoP@UHve?E*lMgzXDykoulYWUcCA7-ch@n(Shw>O z^)HAem(4OQx%_BEnXUcB&Q-O^lsQkA$*H~>|03yBnq|j$55EbHH2>Q$5sPqBTNN7nXKC5yi*%a`?tD;P*sLcuW#{zT>3 zGWM3_s6)D%xcl+n9mR*+RUv(BnC_43;MtX-Y_-u=H+<^H=IYS>Xv2Ih$i9ukOI!BM zQCceEltE3y)t1IO8|xsme1+xMYjfD&z`mV*y#J<*b(*R&;m;9ezr}MS^bntclyCR} z@#!&^+eiHyD1SThzv)ue5SjZ>8uzIh_)fbt@FqK)xf6?V*~$?B@5d~--!pxqpMMpps*pM^hlxLur23tauN zlV%EEa)xlDeRYZKJ)1@>mZEE>Xib|h@;2$d$_d&_^|oCvZ7TLUXtN_P-i+CsfS=Og zZQ8Hy3Y3GxavNh}Rol2Ua!hRS^(MZ_nE2@!d_9hKG*P~yKKQF+;OkQM!R!ryJA9}X z?ydxPrL<==XR74>&9J4Lu%-60-(w?btsnQKT|z$E_)eM9PiwO&o};uejW+J(*-Do+ zYaRH5KjE9q-D;HKI@(i)jQfOtw;5&W=S0f8!1J2u2mC#L+k?N7Jv$EmzYpKE?q7x% zgx0yhD?F-lt%i?vJ}|#!^G(Kmjo_{Mh%pA70UXo0WNnue{ld{z)Kp5yyl)GzA;s9PVaw``9^kCzryS=KzVbGSfje&BzcF$B~MY(;~SDEz7TU?>aaz%65OaI!Mq-=e+L|ru- z{p-i9Q7%=1p3;RAVRo{V|s#UZb4`Z!qh z>Km-L4a~c&N3Zhz0q`a^lvT%utBf(mcIpo-WGsii^55Pw|NSFimOb7l(a5aJqS3Pv zb-B=Qe!f>zR}TLMXhV}d<09s^-*FTtYemqtY4b#HuQESR>4sm*6n(QgqD)=iQ0A4C zDKY)0a}R*S*kdBQ_QZMUm-5t8>lemmzx1lBHqf75e8aHeW8u2}aQMQZ$S5asD<#XE z>sF)78^RCmsbRlB=WIov6Z(KTp#Z!+JGHjuKIWNcfVXwDTl%!j3AuRpb8bm1{oy$C zPV1ADgI`>&gm&=HUgmcUT%>7VT<2nZoXQ+ehHohLTMc^!q>tz_qEY5?h9V;}d^@mP zrm-hKOj!+-AE0cj%sTdMNIdrk7?(Q-K0nyWypX`Z`ilpF`@uc!vX`BG2m*5r^HLeC zBgp~2q4=Q%UX^>Az&n+>9LhM(+?dRN8geS38-|U*P$)3)-N8Qr@(T>drzoLv%H5l< z?D}G=(Vh--pTd6nsI=*et6d?HD*?XCvkohE3$TU`U&0*cElNvvp0c%q@9nwD)&sL$ zq3_B*3+97Fk2;=Su{Lm?67raB>SSK7D@O?}Q=MJPPh4NR&?|wv=|^g6*QSe&ek%4r z$9e2O1s5;R())2x<~%zlTm87T#9!{7+*E&EFyKLV5bw)Tix1WwM{M}>MbEtBs7}(x z3%>s}$os{zRBw|#ZdYIpJg|v5EBN!sBv&X%+>`kGB^HCoW<$1FeRDtggujCkyzRyh zD|zMrRfofCEIIHpxL0YP9~{`g@AFr%zml{K*m!>ED>D6A(jxnF?RcXJ>JMKF z-Dk|RX%(!YmbuMB{yDIDruV=v0q{ zcGuxc*mrh|t3_zlS>OugVpra~EOnjwVCuRlN_v-lEN5yc&aMXTM_D7H%gqRUe>Vc( zjfcaPN72oV2}_$BA1!V!f&Z4#4u6i!1-ob6%X8xe#D2l15`LR( z4s06a4UEAa**w_Wglzs|6EsD>1U55&I&WZc0NT6iv8|wu(FdK{g(=8Yd#OH`=pTx` zo~TT~U%4Auz5B?3(v|?cx?q`O-B;KOVOx6Fv5{-mzK#vF%s#BkR-{zcK+``jsChXp z9G=jYGSsuwmguQiWXRqF2{x~5hh4joIS>cs$vH;T#>~|Bf{kG<#+jZyW3Iz!S#KM# z(1nbNT!*obJzIGz^Uuv=AJ|w~i@|pB7hZ&>$7EbJ51SgjF5kO#yWP$<7-M{6>2{feXsv52X=5Eh zq+Vx3V^HS*1MI=7FxKCR?fOnxu0zxNQLRJ!guWTJIlEdjmC6m|pQ5T=uk-wD{p_H7 z$^7kAONzD^(YGWXh3v@_Si~+kl8#JDqP;1-;oDHoK1lN9Vh369l@inF8^KqVV8GXw zz+N zk?V+PcW_NF1EeF=e84sKh&bIf)*>92g*GAy1Rp#q}Gf9WTutDAQ z5et7mI}~2?JpH1cxUW?&7~_VDdF-PHSL30X&ZmA^_ml41%h*C<2z6kC^!kQ;??Arq z3yzzW1;yDN`!XNF(K)FZCvtP64UT7fr`WN{GPXXkXo|n6x zm%eF(AAU(*OBdV7C;!Qsh9%Xi>fUWeY!{}boh4~d|bggnWJojNME+{>`~z4V@TEIo$lnHhPvVd2)i&04MYc`w-5Sw0Keo8E?LOM}jNUd` z51sD2hH*$Vo*B;@zMFZ9_7Z-}&EyZfj*se#p8D6ELH&)9^?z>E{~hXon13C#O`&Zn z_|W;PDoz`NJQ6wXuV=pn`NZCT=6C9`6=HvaGh>m>u3BU z^OcSLde1}WCMa9)xJX}{@OF-K1b>U)Cl4NS^L!okEAUDHeHO{@t;m9LhHUHUS25XWa)r7(qE=IO35*Zf(XD#bT1A)uJ1~`S((;>5uKOg=JTG&yOjAD&r+V0Qv}_m%udS5qntwV zY2JCE*2%m~XMt139nU*9nBxvvua?O=J=W(5Uj@u{doo`m_OcU;I_$)604Mdq&T#(_h}!E*RR+8O`{A`c(qdm;Kl#|bb`W8IIeWgo^>5nj3n z-|@CjxAZGIa0zSW%axWJx|Z${f8Z3x^G)cj8}UaEg?7WhkE}Nrf8apgTb;~v)hBqv zQz(-uKO6fzY#4>cJXmulu z*gZ!Irg;O5B|_NQ;+GIyY|w2H)ySKhqCG?2Kshj^XL}?Lm-AZ1+bOd7PSsloOyd6x z+lJ`=9+`ie2i}u_Ti0X$B=i9ONAOvR&VB%&iQvfrZo``loC=LQ@RRH-ceU)l#MSc8 zA6;Jb0AszcNIx$8WA4$|01sBxx0bl^RR(fOL$Z#WHKX_{WzA^3)|mvq$vR2l#ajIB zLWfa9nHL*|d}F=ne)4a^kGmsY`};j659(`0@h>C4_VZ1~z<-i{zT=9X`2+Ibid^m3 zNnD6*`ud-YwW4p*Mp@_VW_^EE0_#LsmtK{?I?)8yJtweEG@&1N)`=$AxU)_)!OmUw zcgc6z-_@^Kd_vevTfd~8H|$rnW8gWRy>`d!YG~gR%+C(6>GQLJC)Ab>bdJl;+NMjC zRr`4E7*N*Y+F{ea!~4+`SLiSBV>|J9#1AEWX|q>+8OXpNOy0mPcJG($HMt%?{9JfV z_C<)#@p_(RzlNLVmUMiNjBiD!gt;4WUL}6(5XQ1uzH`ti!DW=S2wec52+YDOu6ZV} zaF$@U@XAu*l`PKz#&BZCC!Vc%vzgx|29*|A1`X=@UW$!By3`w(VGD&Avxcs=g#zQD z(QI3&a~|v8Z$WttyseJ-I`{-CWSJUEIXRfE5kU>4BEVAVN z0b)xEJlDbxBI6$dSHjO?drRN{DOWc9S6q;y?_tsVjhf*Z`k>++_MqxLMqj~R()I%L zSJ<0+zo8HB`#4;AFS0^n2#PJdtU!6ehOc>@$O>X1h3|HHMIOjn3bDaupO_zhk$$zE zc>c1Et%mvBCZ}h{w{ncxX6I+BCDnSr`mtIh5uy#smLV9@7P<;zgEnQ=lE8DzeVgh zc;cP1#w_rVM^n3pqI=+Rq!sjAIUvh_dfz`*S9iX4!>oClQ2Fg z_q^80y3DdWn5*aflv>6n$fX9J#dmk4JYG9eFwD@QYO*n(AE`qr%hI7u1-4y6tLN^H zkYBtD-)`cY$glM=__jpoLp|cdn##YL(=J{L6KLb9J{b$p-OZ{=AUBUC^@=pGyi2Pgm zHidMZ_u=tK-nRjVclxcx3;6u3CH) z8b-#;aDgisv!qe}pH2Ig1<$MK?-Rt<^=#pJuAm#Y1W%by>GzdzvGI;o!~7&@PgirY;T*{UTK3B7h~ zGV&)&`baZS(yQuJZRY`Eth^qljk<_@kDK{c^yE8nspN~(-ViwNH+;z2K1`mPyUc%Y>-qOzl0Mf=Ctj76KOkl1+qD-cb6Hs;XK)z(NBVrtRp5fSot+7# zzY@E_h27v(KHn|R!dF%7)A${+^FE~iJ1gC@#J47WM&<_ez7qqNf1p0$#q(67&5xUH zW?!-Hmy-WP$IHDcKTgXepR`rS@k2du|GJNMZk6`DZquCnXEV!@c1d5q7{2dp?cLuT za^{Py^MjA1?>Ta(>V2S9cpqNMC0+k6Jas^y-|xnUq@)FyouX6JqH_CN^$5(*$RAslq@3G2mM=lhq$P@az`so8^ zT(+M01roP*z@@?uz0;(B={!!m+|<90`YnEy_?(5jj{x_R!JCzD6zz}9gAKAu_Ek(U z^9T-vUsGwj1*_m7NPi1~htU!F{IsX9e9|UpXT+g1+w>2{KhlThL(446iA76+Gc~u= zyC`4LGN8#`;1Rq9>I|A3E9;>Nek&bsf-{i^HW^n$;izqLY`n=jdnsGm5Ts12d}-@3 zY#p8|qBQ}B6KW>?dY>JA}Dg4#_ zpQ*#1;=fqBVWp(U&<#J7^pT8@rTz%rVAcDOj*A}I91X+YHyLu(%73=me+9SsW=zsA zp_?~GyKhgRZj?N9Lg%Q&VGd!Q*_ElwX;TI@(GEiO`?oaJm#mK<5C6yxV8ZlgXwW2md% z3$Kx~tKfD2Qbl|BDQE?J?W{)>Sy_|FIVWE$Uf#DZi)Sr>__&Dm(O^@Ehtv}b%v$rS z!=?<>)&iT%2{F&|q~c05#%lQEYoU81-*TBdvhX17Uue=yU};6h2rRz>mQ>0}6P^-! zns}=L4oglkHZ$}c;|e=?3eYB*t9psJL6)8ueEhq~qt{T6A~dweiuGgToHpV(LBn?9 z@JXGHDEdvKoQ5d+HN3A#9D>i4h%rwFwEMuM9kGz^Yy-cMw3`JEilS&Y8rl^?H-T4Z zmk~v~qnALtDGIhms_*LOl~q!%PPa=G?G|t)aMeIJf!Ph+8lamCx~&Iak+RQ$=^nFLv+4vvID(vA2QvW#Wm$9pSv+yB!u+jv!T9e^$gBIBJK_--OS3-%5WAeKAMz5{t#F@NV^Jpi0`Twt#SwH z^0Ja|f=8oGgS2v!b^-9>C>?ChB`A8#l5kihMv3$*{Dat2*O6D~X0=J$V)-5f{_((_ zLD{d6Px`LpmASzI;2}Vp^|1tWooCS%A6*r66~5|KE^V}AvXuL|#MGhwBhdL7%9dCJ z--OnqB52(ZMQh$;(|U)rS<&)^*7)}&){M|v_PhyB1P8Hb{%2sd@MGy@ljbe{a@L;{ z2Z_C1E$m%tk+%J0gF*8T*)OllV1wo@{sHWbDs!_p%GJ`8p|qTi=DW7{P4oBo7D@9r zc$fZqCNzJayuD}>Me`SdTWEf16wRLk574~3FIQPKr!Vhk@4@M4ez!^U#n60|(7e7U zmZHQk6yIK(#6bCNy7nQw`>=bkw!c8uVmN#onZy2Beebx3(Ke`x9b8Kvt%8pE=*eBg z(w4D|Wt09I+u0Gv90dCZs?oQKi})}@`-lS_kxydOKEvA5>fvVj`9}FIQvTeY7`Pk7 zHm83`yxPB-_QYh``ibB>+GhWM?CW|QSbO=T%yB}4q{r}e)sg-`@pauL<;L)J-AcOn zx-Mk>Vk}?R_vOE+v4HrxZjyAys!c-Q2w&Irk{)9`GfUD(n)yffjHj+7U3^^wKQeq> zw-T!(manT!&&Rr5ffN5vJM~8Ux{9Ta7{0FYl0Mh08^47$mK!T&;_Euf9@TH->q?}L zeG^~TAnFr&w1;uJw0X}zjW(Z0n=QR--Nn}>K8RSpt}roBVzo0wy7;=9`OlK=mal6W z{rg<_J~|$48}_E0cbvxBJE?nBKJlL!ON(9bG_hR8-bFC9KVD54qDz{GNNihT>tPE@IX3}Aa#Y{@-}blt7yDjvX@@0abid^{^tWvm{KoLN zZ4mt8Z`(1B?|t~&ek}jS@V7lB=~tLMSk}XXEA=va{B5(#&)nbkU>|iYk}~nPje!q@ z4hnexA*;qpdJI`L zO45;40g-JHvdU_EPH|M5qHyYn0mI-vVCXODH)9`z_xZ@EQ=Q?ZO5Sr#9vaufLtn}K zz{O_zr9J6KB;De<={@P6Njf~Lx9>*MP2FR)@8gSlw@=z5{NaKR3W3w%?#&;y$O@sm zf(%&*U!3&ap(o&1>rADU^y@rg7t8rcxk?WC z<)1lw-^4yCD$Ue#)z-<1>Ym)4RvRO&Ek>G})$8AAeoCY5yRbDlyV|?lmbg>qI*P!< zLi)^d&N7?MnO3)|X*;K=iCym6>0fRH7x$31AfENg!-->Dl(f!0I%!=YXBgQAId|R( zzu!ZwWee8=Z#8)x zDBNW{QNCK%W(+_cr~7t&m%UG{ca*u^4Z!K&KR~y?<(~@v*^iAkPf67p_c7PXw*dV# zPjo$d%w-NAAG@}VZ^!r+`EN1*Zu{FM-k0i!1veHDGl2i3K9zs>14AYIh$VkO_Py5+ z(6-^5?EIoMGzFf*KT#?0O@psGkCn1U&HYr7x*#}(c!mO_&R6({-hIS410gz3t;Jj9 zxuAjl$~$t9Q78M>#kVIg{2mw_v}GY@dFpA@U4d;Vu>x+6NPCg-l;qR%fY*Dll|?^V z?|107bE?ma?7E)+uH%YHCwq2mI#n~L!oLc4q0?WVhE4^<4h3(wMANChzdvXC;+sXMMPsAr#5aG$zklN2Q_<-&{+IkE5_h1WzcxeYMEuciI+3Q+iFl-a(P^PY zr>H!Ubo%|H(8-?NH=TfCC@={ARf{In{SRm&bUNN*(vGw%qv+HbMW=t7beclj|HXfw za^atNC(oa%DSpM+#~|;m>D}+qG9Yq{B4dw7vQ8d*tYr>P#vYGgpNf97eB4$)eri(B zSVYEivM%l|V6pU}mA`>pRCcV{_@x<<~#aj<`&1yT8l>5;k z2Wgz^Cwqhsy43Y+rCjA4SE%6f7`i!X3^%?H_@~Bzzs!XHkH9bRjm~;}x9^#jLko&_ z9$N9w&ku?GbuL}Gd)k}N9h$bV?NHGxUma4^X)Z=zoA8yCgYCApD&H6%ALPGC-uzU5PxCDXr;=az_g%(9!kYy#c(cRI ze=GT8;Vyj`Yw}#Kmj5Q7f`35PJn!Kd8NXG^Pghvm=-}MqD4e}On9XsG7iJi2G{}k*OY3EOP#(vh@>1SQ85~mP@8h#td`lzV(-fPuw&XIZe*OEy_ zdyRRT?l~H1FZG-i-Cpzmr6&CGW_!gB_)lWZMb~>>RJ}hi{|iFv&6NFfF0s2VxQsQs z%(q0I5!-3{cku_kllO9l8DGZf+lY>T_#xymYtZ#?f5;GD(K`<}4%7VjiuTXLmLSi_ zMd~X3d%*mjdzLn@PSIZHTMpm0b6I-Zn(q)Fq!kC%(Vw*dMJ{jWvv#jz5dOLy!(J|w z7)I*(T4TXvuk6oae&+L{4XIkws7t*~dz4*Hu?=51pZ6)=05Ca9OSI+GM_dAL(+*|V ze0c`$&Sf?)d*{9TfOpsZMjTTserX)Iux!vS`bMM;+FL%WzWsd92Y<46!;<^b7sJ$L zopUIDwZ{6KU?=d5$s0~g3}sg$^zHnm;yu8+M;%8>rnW=&S`N$9J|f>6jb~~7u$_NT z+Q+Ue`UGoyf&KejHQ-+Q2X*MOyRUPEGQl58XKPv`vR&+XF2QRc}y}uwgSj zpW3p!z8L@5+9~$TuIU%7TzkFjmBT&@lkP>PWi0w( zc?##iCObSC*lbeXUdq+aB03(feAo0j9K5Ngyn3@d%f}$&5X;BFhAU*3;bS=Zk})r3 zoonF zmh+W&PEs8^6V-Ijv$o8xqvJhmB~C@r1uNEGM0`5&pE$AU(`Gu?NuOSX&R_MqU2kV? zm_2~}^Cmh&u05Bq|4|AIR<;_mpk zxT|ihRkT0CyBm_rPt>V5ZLilJ8v6)kxhU(?m(~xdEB!}F*@q)+9W zI5YW1&P=}XHP!`|6aT4Bb-mhCSL;c2)jd`=R=!JpRc3v4XBq!|n>y=R7df+(xq8~Y zQdPAB_GI4&@I(=NS$xdL%;9-H_>%N|@N}y)+4C?k)-Ep_TTi`iVE7lXJW|?P5(oaC z`EbN-kARy$fWwEu;WNWVq^tx-xhBj4yWmJ*x8Pk_d!^3}j#h%Bl_rijzk2ja{mL4* z5*%^9^o{GmQ9b=h=I#264aFX>h0a7zHSi4lP&scs{L=<6$MGD`mB7`H%g$xvl6i^g z^y4mj^5|>TB%jQ&Zd3dFWSzs@ey&&7C$kTY|K~Yfud1vGjV?##nWNLBY-w8;ZIZTq zVzy0qrIB~Rji0;FtChR7-OXLvuH@$q_j6yEJ!tg8^ns%n4jM3eVRFjoh0f&B3;XpS zz3?n$^g?)KA?;sC`=#B>Sj!6ER$f~=#v}b_1Mn7d=FWZenH=_w*YJ&fHS7a$_~z4R zB#-2;$w0RcG3L=E?+5h%diIXj(9a|;l%Kh*4dl~xd+umtQL0Zv2a8_LOiuT-li$mC z8~ztRvP#YtnU4-WY*SjogO%cT_I60U_fY40^2)w)S%18qvb)Q5kJiK|mM7)X7c+>@ z(_od$`qS3IQf`KCHs$)6kH$uMmH9YrK7KEWPg3)9o3{r0P-H?Gu-uz3zKPy)+p9P9jRxKeOp<$sueB!7;X z|M%?o5c}rDQaiC26fOB8msieDFDi6-FF2>vJ4YSVb#<+__8emEe@-7I?hk7b-Mc-^ zW6HNO)^-+VyF${BH}c$^oZ>mRA3i+jxNCmR%jYt_XkD^kdkQoi1TB?#<<$?=2}Bk6KhabC?GFXvNknKH=Z zT3oYT=DXWerKPo>^6Eh9d7Cn&jt%;nTqUhP@CwW_2Pk#Oo(I`SFY^MNQR0*KI@SIj z`a5F?2lh%i`}$Sgh6q3tM_Xacd1`o>y_QALM$w*wF6-5lE|LAuC-?VAeM!f{+YUfi zd1gOd=BZ){%DjWv?o%#P4(_9!tXDedJO{nSzsplnJu7IJ19(40e?CiF-LzF?je~0r zeXwyf`^AtkrRWkRH*Yv=+$v|Nu9i<}&q`ae=W*2)dc2Uc19_WZA}*t6j4>3f#dxegX-E7mTH zA9zr`VENkop@A{ zDY3Qwn3Cj6B6ii4iqmsza{rwx)771W_?}{S?&R#Rou5(GlkD67ZOR~Dfs*C9lYB+Q z@|phmgL{hjem383=KlK>hcDZ%?kpnz`zcP}SFCe94n8x%*YcD^PcC?owoWVctX*;c z{GeyS%|XxoMZpydMh4$bCJsZjq79^vT#%w@4_7JLe{n5kPu&~Xne#Z``HbRfpPI%i! zKFT8)q^Am9;;{|%RCAX$?kroec6#aRwR%0+B}Y}eNAlB+ z9T&HPi=E}3wMW&VIy_~>)slQ0fpN7h-qQ+Pt0`Y_BJGGnp04NnT*~pJ^w(h%Jay4u zFXx-UwU2*S0*f11KxMN5d-2&XfJs-(gtkoZhTsJuG7;EJj=hoBi`%^_+kwG z!~@%mxYhHUUsJl zeo{R%>0c|$h$TRK)+3AdVOL0;jyhm<(Y|w+EDSzgn-i3NdG3-i!G+g)*4Cj@y1!BX zuJ8CG=A^;9SQdo=yO3!GJf57p$Q zBzVbH>RvmYGS5*HJtEu0=IQ{>90ghum%8Eps^Ieb@8!x3Hr#(NtR zaX9~``tFjn1T(EQM^F0!X@)J3%fE^Gzcp4`RSs!MdfGhF*zatV*Lt@8?@gq&_A}EI zqr4kPE41oW&ejz0D|OF~Oa~qX*&^vPNjKUjup4z%kX99E)n$}hZspTqGTL~lm6kiQ z``=>H+AWx>dg>n^nGUR0{bQ{%8b?uvQEs7?=9Y5%>+LVF(v(8d3_K0D()^Oo_&3{1 zb4htdzBJNuS(Bu<)1Rxidw`YCZM-M(9xk`HdkqF#X^@_7=*1hGW=<()VoiPxfRVbRz4-j_6Eb?>KcEc0^`uJK`**Pdnn1 z)7TN8M%WRjYwtJdn>KW;-d_5q-d_6V4%#cRNVD;M4oM)s5^a{YJ4p}N3w>F3hi4M< zW3SEOk+>D;?`w!jd$9gkxH3CKnP3l>@9D^}d**(3$hsW6dT=gu@guAAh#M*LFmT=a zA!Yh={_swoovL$sZf^eY13csJn2uh!vtnQ^x}7!dgB#X5l+2x9Day7a>VIacx=Yp} z{-eLd+|1nhH+%z~z)_Vz`KGM0D5V4$0JYh#wtt09uI-`2-yz zI{W|MIwY&B4B6d@PB=iDMaOp0X3?=SrmQ~{t_;#&0^DZ`{m3J_NbrI#3Vjp22#rqC zt(P0Rwfh@ocI#HYeN)|PmD5YNQcf)0Dz-&T-AaCmDNx54S?2?EtIh|yZq4YLiyjyL zmaz@Iyz>~igO_(+g-v3_FU->3qdo zV@vWZAI(_+gGH}hi42>8ESru@n}%$wAZ`w^%*u^eX6ykWF7d_0pCOjSCe_o)6dn0LvJ{4V@>iZe*g>j|NR%089uaI%qR&a=4nV6}hi673X zupVJ?O_07CaxN|mPC-T&o!_u_3^vJ((6TMd7*Ca9bIfFK>T~GC^}N3hJ(Bw;c^c`z z{rmU#)SCVHR&1oi{)wI>(p(Yt#t8CvK&wyCpT>8?cDC%1%ZULu6CS^f>niy12V9R) zpTOV&h6Ct0fy0g;_)nA*%O5K`CX@DLFdlLoe{fF?dv_#0iTGQfVWrH~+{YNYgf$Cm zC|CTywcIb@9X+&Fe8|VZ*%qGfBgV-%(ycs&MxJfp_Q|r=BF)Wz;yY=1p_>LzmQ~5v zSI#fU@NFSoV0oH+TTXNO>;2m+yzYOBP!Nk{!)9ksd^KEErQo&sS9&N=Z9$+tx#L#br_hp|?2WKGq zi9zA#Oiuq$iEBe%;(!-Q91~*dI{3E`eaRezhR%4WsPud-usK22-ao12j?oT2hwPTQ z-+?auY*GA;Pp-^6OY7Q_dKm91-hcga zfH%Cw;g$c~&25wY^~*~Fn-jfVV}=wpP9;8jflYgpI*9$J9dRnmTg;+QUc=l)obGoJ z+R3+n@lE_av%rH2Z=Vhqdwu`=xcJ5Y16=&j#Ko^pfs1ENTnqpgzk%Om9>6*$0)Btq z!TeTZfj++ljmMbhcQC(IuIws;r(_*{CwmO^`7OJmW#1$7MLlOegzF_nsS}x)h-^%P z9@uzGlR0OOb+OPx))fiANxVimle6eqaQ&XkD{;iFz0NBA?ywSkF^j&S_Z`<{^2=Gt zX}$xT89srvJ=e1a=px{exsW{WGkI^sFLR7K#3waL|0ew^9T}hI8$o*m+>g-L0`Do* zho1^KhR!wScx2w=BKSIscqI-yK3V*d^}wO?a^pGLr4jUdma=tTzAy(~wtGF;PhH65 zE_iui^*P#DzIVOU&zq#n>jZDtBw|o)QM^0hexDfhc>he{X!~|>Y?1O~)Vuvm>s8PFR`htfk9vQ0ruCM7yLuN< zzD|GQ$z^NhI{k$gmk{?nk{=&9Q#igQ{ja5bp?|J@%3dA+al}qAVuAGP7dM_M9BtnY zALA*%iGEzeUar@;kQdwPkSk(?Nm@I7*`elYZ<6QU==AJdt&?<#R~+4+&pA`LOTQi8 z_M`lE6K);G|IXElfcZ*biKJuV|1li<(V@NI_({I@CUA_7fs>EPduB4_+tIR>@{cio zT*Mx=fJr~0X(fAQW&K|u5?{4LH7D}C=I4r6@@(ZlYu+fId|O%5cw*^H^xBo^w;AZU z>Dd0$IM>s#`^Dyn_218BEhIL`1-j0)GxvV7A4GJJwf1%*x@d2mLz6x!Jk*X&W<3`& z-q}X_)BQOMi937rwq|}0*=^>%j`wa_yUx>|IAi>NL!SG+*kH{0QND#Ek=YiG7l2z` ze*(j8X9|OQ-Zzt*CJa+4-)esm{t2tze>>B9H+;K#izr`c@$MJlZLwrSAvoxjZpwyy z(j#d!@=W3UrgFi=PuLsPAOAW+?~lUg(PLR_3>z+w!*7=afA@#ali~Li_y!TAo-FCjw$G;)XuA+Pk=iX1lm96YA zkhWYGRqu1G(_1OFc?7TQB=5r7zUzIKe{|bC2Ctk?`NF&3IndL`o5=4l_5V7vFI(`8 zKT~+Dz8nJ&r-G+H5+Z5 zPdZq(@i(mZ?TC8c+}~60duLj&WgCCPdY`9!>Gw7MDEgbSd?>j0;`?rg#~jc!R==}w z^c-dBG&@bdyM;aMy=ZLOPlmkyH{|fu$m6S!%QKPBSK@n|f$!1qdx#w9hNm{R9I)VN zI74{Kf#)USN?)i3CuTD)f1S3LRb9ZiT;g(O`r8lgh3hcgnJ_IK#44p$ojw zgx@PW&(HxOo_pi`R6HlJa}s>rAO6N?v@``@rCmGBnV?5F6SUiZ zRL+_e*4qZk4S%JqDVB3F#E%j9ck~$FEqhe^IsT>Vh#AG&dCR}F0U!S=o8e#5`8dMA zG>db_#XsJeTKe+*!OF{>#C@0_PuvH5pjCCXB@$Djs;-WETZxPVtBA+)40slw)E3$k z>62>WT_L_qVZtiBC%U zvsG`9e!t{Q>nbvx}jxiHE=9r(jKcrTF=JV9NAW zo+(VVW3{e1r-$iSFPJiYV=2D_`&Rm;tOu07D17N>Eu=reJvj?_goX>*!dZqd;Y@Lc{OJ1k)8p)DFPvrg{PaP+Eww{*c_I4~BHQxkPv8 zy1#=sX~;+ZFBX0MA2$A*f;~wrgTVjC-kZlqRb`9*=Tv2=$^d~33@TI-lmrm8Q3eI6 zB&Y##LRxLLr2uUcA|kC`MHCW(5)xDj<5YA@0Ebi{3|?qMw=@Z;9THR!uWft#wFzpQ zfP&zZAave$?Q>32DJqCw-+jN|{k%W&sho56u=d((t-bczYpvaDW#4ATdms4>?|(`6 z-#dsH(^u%rT*vQX4yCT2Hq(2YO*~Himdl zdvH9DYLy34xjUM;9MSi4IYT%kdo7_M^mTkDRLoAJJ)?5kLSl63TgqAVmP7gdl>do* zy60!$Q$>@HpDx}{&8J=B{SMmyF?{;&Pvp~isZZofZ~ zPs3>czsje!(}nRT^67t)3*`Tke45ocR^P$;8IPY6zN)SF;0LCm&-?a~XHETIJ=Q_o zqyzs#ho-gGN=*1F-K#s@m5n7@W#g08WA;B;GiKWI{4on`TI+%oZS?fz_ zr(6EmG?5o69WioNY@Cvp`4@PvD#sUH)|IkLygOfz^EMgi9=EPw!Vhr*-|-vdzU|I_Jj6+P)AGcJhn%E^M!Z&V-bN=lNCyXl_`5>S zcy}tfIc9EI>6=Pi!BTwm@VVTQj-O<6q}IBMH8J^vq2AYlc~Qft_WX`1$YHTZNx7-y zx6s*BwUnpwA@qwWssM$eWpWiTcqjy*VL zUdD7b`8-=y4|4HtP|=J~VD z!>}`vyOOa)@tkCS9faQ?F>hfRWp z4yDNYmsIgRYkdxF*V48l+)*OA(4?#l8u^@W!EMSol@BR#4U+5dgXz+CQpJa~_bql* z$uZDXdtP7`8_Ah|y|f;jhQgP`LrSgz17_kH=%>ttxh4SSBJ{d!_9GoIhrTo6UT|=_ zWAF!zLE7((6Qg~PIfjGCIUi#X`6@Wsk1vE{k()cs9Tms#(h~xo|2A7s2;^sYd9>ug zsDLLY)|}xj(X_Zpxw)hAoG~RHz5n}8lgAY=j~Q&rrV9*S_FRZBEb)r)z4W!8d71(r zFNbEFafWPA_0BWpk7ty3ihQGv$PM-FcjuaIj^LiNj&|~;Tz-IjX&ez+zKj}T$QQRM zUpymJehwvLh74(!G3N%u^nuy0XJmW7(}90F_)}|s1wVfW9=SuPQ$2h%4LR3-Et{Usj;D4dN8phLU-Qez;;5;?>;}csD+n%T9 z=U6ia=5}5)y2|_V6+XM-r?lXchmX3EqxT)p+E;a8-oEmGKDe(>Y+}7mjImAYQzB!u*f~mx%YOR&Y+K~og=nFE8 zm~x2~kLA1EeHTYZzAeKv775;HUP zJmT&qYExdw*S%%8xwowXMuES`{fMu^y~uY3vH2Oyfdjb9(ZAPvw5_Xd)3$9!Hvid8 zohj~ZtHG^CJd^lh$3*2%E+#%UFJq}MlIO=VpZBfJ$*WqIGqXivQr3mh#x>g3vMJiO zKe_RLap~UGlib_ZBuGqgYQI=`KbqWlGcz9Zt)-pr{68zJz_*?@R#Sg1?W`sL!TM}% z>*`6`wk;mbTh*X>*Nky*d*Mv-Rk#N9i^W#djaa)n+NjKU+!w*~`pjp2l{vGj-k^>3 zw6Q)0x*4TyeQ}Jo?GI&|x5C5ynYYl!5TlJaa(6kSxo<9GnJ+GLiEl6Q>Kn=DKpQOs z;El=zt##v#+SV6t(YC!>t$DX<)F1EO_R~-FJ=BS$&fWaR@!Q}!ub&eSczxGq6@&A;s${IpU(4D! zaBhy*?K-<(5Ab`4a_470;fvsZjX4tdY`{momi1@Sk2yyg_~;HUG-#uT`w3qIvD5OL zn)SS|EXPxIJ9VhrvdN{jmO&dYLK_~&qG<#By~MX_V$;g1eLg!pB)H3~De|p_7T37I z-2iQy1Ker)z_(fU_p{PUYZ@IQgw*3G@Mu6POnV z^bwO$>e~pO*X7(1P?g9{SnMc;d32qn|sg z;;1(S{H$w@@-%)H&Xs=%5NN^wQhhHU%XA*)&PE;%->MPaCO!*%)?z( zjnMg_OrP&<_;x*WC$zGPG0Z;h*|#6qh0gPV?VTM3KFJfepYq+I`NK7j($DOwS=I0k zG`_){!|sgz6TxlDfPTYv7W+P-{F?FO{oQvI`XZRiNMPMly;zND1M_I{Mj1Svz}S~% z7WsB&EmGfSFduecb?|;o&Wx6gjAJ7>4TpBuq=7rew1zSs=xl9GiLbKyY2QoqSBWf= zd9O@k-iK)0zM#$;V0sb0mwqaDFb|abq9)&`GpECcRhZ~rWHRH8yPf+Jm>b%46 zXZ)5y16OA~>-&^?LdV^~<6-zlaIh5Fu@2mAFy%x(aNV|i`yKG?T6d}MZtCYTekFq);44N`e9?nBbHM8kcrh>QY4Dpp0k8wpKZv;M(*Kb3o>lx2l=6@A(R^}p_d9HUOlcD#G zlxd)!!Qc+KTGvAJ8^LiIIL-&B?;t-u=6iSO;t6;}@Hvs(8rM)3cv?3gOV%Je*C?5N zUcX}aEsOFM>`yB)ihU8R9dEe{l+3PxXDg9eYmixw0s9NU9?S2G{N@AKtSq1J1?nz^ zPju$5JNRhIDDahoi*<~@+=ZMTplz$APB?W$e%0=H#%H5Wd4jw3E#yhsj(nd64g{Z4 zuN;06xgzC-4_3oFF6yN*m%7v&fKT|2BHvxri+ugSnaIlZ%;n?s@dkCS;`dE{%bC+v z%xM`s`Z#kU>rXM`Q*r}1Mc&BxptG$nCJ63Q`*nwQ4*R9VCC~t?J2W1vQt23))J>@dr*7eZxsxg{cufv$n zXx8{ISeImdmGyZr@6Kj_GD&Vm@&GG{O=fGbezcaaS_#AsJ5@_{hm=k#%4678&y&+}x>Brg>K*%Psv$ho<^K!)dI?=enQsy`8zJ zeV*1+e?4;}@>Ar&tNEI@+|*lM7$EX?K)*Afi3I3wR#vI+3F!4p@f5)6XwK5NtA9gEbuA!`k1*;`*^4%Uy! z^jAWQf>)utyRr&>cQe*a;9b@S(G4tpO605HUDirjFP8Be!LJM15(h5NXC0C{=TK)I z`nS-^MsO)Qk{fuh1dn^%K6D9VomP6R%qe)>2u@xACwt7c%Yfy)orOMc^;4?NWZHZi z{#po<&jG$;jP+_TaPDT#WsS+F%;%X;8#0Y~ zUkj|lbFxm|PJ3(l{f&vgdT3edT#X(i_v%V@Jr7j0g_JRX7{q)rXx*OO5lTO zgG-?cwO@k%RzZIPyOb9?sjn{eO{_8SD6+nSoTnAgA{%3c$5r4lit@5HXp|qc)8{Ke zKNA@yvS=;qo};G3R~{hG1|fTF>@^3mKh9=P+F!RFw4E*azHA4Z@$J;v)5=~lD^;7a zpR?QT;l8M-2*J|WM;`xw0yka-|+%op{dF-cUZ!dZCWUnqT zObf^XCo=L7Id3YpZY3Mo*Kc57PkubLuU{3guV2r1+1G!A-Q!#QRlj3z-y3-;`|p*D zfvu=*${)y^Qb^vEX_7Z(QtuTj?fQ|`?8`mSa58$O;N!>SP4SR71s|kqomzZ-PPa2l zMw&5g;`jCmHkilplMVd-SM1U=ZO)Pg`jfel{vIKoY^KfKn#Vogc{Z(e=FIe#1Js{D zE-p_u_qJH<0C$o@Wg2xt+_<`ml&he;lR4JN!!?Z@6XUcq4^EHQrp)c8 zm*gk zGTxcwR+-6obIGkD_9gICe-Cz=iQq%<)o4Fc`NN)=cO$l+8?XUgk1gnz*o3AKn{XXx zQ4!vQhuV5Y(cVJISLo9A=GkIO&V{xPts(yjZ5&}fT+Fv_(&aE@lI&gHnLDUuB7Gjd zd}ZG+vHzuCd0vU*%3&qKr>o4e8<6L!Z0?Yj!&xgo%!}<_a`?KHA6D{9uC~47Y*7c3 z;q7(sx$yffx;CP}9&<33yo-lMuIwxF@-T3GL0<<3lAq|xK_%%|4kdqU%t4oa#jyft zv61gG)*frbR@(JA#a8ytD#OP!FsFCC=EiS0zT&EEauq0F_>R2Z|5)Dnj-0lglAqZQ z9>c&V=e_eH@P*g$h3EW>oZz*b)o`YD#Kl<`_s1Vu#qX1=JD>AZ19MV?=Iv(25y;t8 za2a(LM49k_GDXaH4(&TXeQ;lLtfO9FwjqB{%)1O&E(NCX%q{snck13R`w849zZV?T zN*tPgi5Yj%`F_e8z9qXa0iWE_S7!K_%Gp!{Fn_S$I0I|If7dz#iwB>b&dJO&e+?)j zaMvQ=g5mxPF>G>9HhDE?jkF{A$hyi!CinlUdz_go4O-Roaqo-mvbr07==lBq$4%Tn zo&cT4jy^(`Q%4 z*16onTRuHQ`Kw93mh#Eua|zdaluy>SN*SHLWX#jp=2)yF&TSGsBOY(Wm zX@n;P&X0go>e*C1qg=uwxwBsGDj?rB-zA5Rlzor?tEX$;ekDeoYE2svSnq85l(Uq; zcX~iuf296vv~@o?Y?8ZFi8*Zg%FSJ=ww5B+_ot9EMfgf^=Voisa^&(?nJq=+Ee?)T zm7GgftZMTxcdPgWG=Fs|_|jUW?Pl6;rtN0hZl-N~(dLK@sFoN={3&Mtj&tZJck6oY z4zu3PoqY4_kgR-acH}8th z^1Gn3+2D^{JpQIa=q%OlZ%YMd&`(QhxT3S$!D$t~Ar_tGNqM0&-tC_Oo$WMuO6Grq z(3u@NE4-;n+sXZ=Fh!1#iFy}e7ETA zBK{XTTT~pH&PwPrn9lA9Xv;-?i_RoIqs;Cb!yOu1%k5`W$vKOnvnX^x`-fxlcjS+O z@2ZOW$-S*z$4@kzL7WGDwFZtT~eJ%J6xT9M9sv;v~;kF>^Q$f8+}bihZlez2!Xd@V;^Q zCN-14D^r_rYzb#!BE$DH_P;-Gz*@`rmw+1w`LdhI0VJ|VWRCc9%3AdC4D!wL>{uYN zS#EW=nZ}wVa!PW*1?Drg{G}QebJqub?4*w$1vBTS&JzzBFdu&SB$zh@!`wWt9cBmf z8VIx88S|gq-SJ~#e)`A4Onx!viHEwx^TJ@5p}|w(dHqj-IrkJW3q8u57r_U?G$QwB z_~-9h(PnoZEa2SN|BnaurSwRwZwB{MHFEItPWVIUBZ_wni7Ci?DvxX5vfr80%sLx$ekIFKd~^C<(mZCF-RTFCc5x zNV_(oHp0DJWX4kPluaye?IlLt40O-GlJES3qf=Yn|E;^F7To86M~MR!{iz0g7Vx}- zwN&2CP`YNX3LX2I$fv11OP;@P(NS`2Ne4e>KU``{Dv|hw|K$BHY+Iu1(#90w_v&M9 zQv~*LkC8{5|7A^1V$Mas>FT`yTlQ})w6z0#W;Td!Bsxxf#aGOOi*tC9pTTgB4ubPS z^&DSOjck(g_p33|4nD~uU;oH^MVodTaw{CUh0Z=Nk~;x(?hfGo>jk>MM)Y3Jl^e1& zZ=FZ(IU=_|@zTTbp{(Vbqfs~YWt(}QyWQV(2KV@=dca*sti$gOm~+he5cw3Z#xMJc z%jx$rbaebV?S}u~z5klpavyV0-aaqf;mW+MGUsKrniuZHGUMf}d1>WY^4T|tUuRI+ zqd{d4@%$AsQtx(nmY6CCS`+nDcC1?nTJxbXV*u z#wP2N=GMG9xmu}{xehKPbYYcwJx}6`YLt&h>%#_~!su6H%@{pLPmtd<@)71}wIg%1 zJtJ#=t0#DQzcIu&=RJBt9q*g@3*{3l`X+MZ%j7RFF!?mB!+#aJo_Skm{tWK>diwq? zZA<^kzYW;5aJS!md+7cKVTZ!!cFq^zWGZ@EK72S8okK_eAH<&afCbM37aH(*B1LX2 zd$S?Zo7zM3ir#S`OuMjC*#TzRhH#^-C&HVGeWqQ`Fpg!6BbBfI8$53AmO=650W z5~-I?-|e{iomtP-8CRlTcPf`{!4=VgQ`$@n!s#jJrP`dAfW8(op0J=c6z=h*7P;DZ zoA_Ue-eH4}xX)uAx`$o&CQNcm{uJbfUa^Dv{|>F*ht9DN{N7{h<$aoaQLLPTjWgt~ zkobzx#Bs}AQd@fB?~Jcb1N9%}U7(Dz{Jn)XUV|P2VOa?s1k$R|;y>@!6T^~|6{GAP+$7!xPDbP^bLlM_G!u$TBoNtae&Ses` z;pO}KkIJN+_-c=HyXP!a>OX2B^;X4edk;hi-*&8J|9>6F?}@_>9EZ%OA-lu9S4}nM*Lc>w z6L}8S;YH59k-(k4wv3hqHofI*wZQD*FYepfwj?pW#=h0j-rjuLlle3B`OKE`5bfPZ8G%7Vo=Q1SrsjPp z-he~)C-SY8wW*vv^vnDOkL@)DE4sI|&3Y0F)?0#LHQ+JqI>B|WHR1U?eFonj$-1I+ z6|rxL59ZfbXRs#jRO^DQ-KV=Ij#7QF_c3*qz_!i@)@Jl2(W#Qd4IL-NDQjRt#h{>i z=cwnzicP>8h!+{(1m^om*5;Kqt#vrKku~%ZHQ)4W&bKv}2_f(!<*jltCQhEApTK$U z$vg>MUf#Qz8+@x3&8WRpz zNSPY+1y73ZwQwgnk^*sJ;cPp7#+kG)a6XIgsrXfX%U<20^Bq04(n#JNG2clHihL8? zJD`2fe$BgrbV8zCaEm zye-`OC3hn#8?t8e_Ej`y#6Y~t7#sJ`HDVuFFMb0Yo=lmG&Ub-{Kkn=|&&*My2l{B4N2XM2bHzH=^PParV#;J&omwM{~UxtY9U9n~_l zy|R{)jJh=E))3`x>Rb;0nbLblxLe;7`SlBTYa+U|*ojFV>n-JcL-uOr+(}wQPW>{* zu49K19q4)LeFvTA3^mtB>gw2gjCH(&W=ea9rXeMV7-t|Jr^^c6yuwS5ZOrB1~L)MRo+NM{*`9a(NN7A#1}e+;7jld7H|h zQ(40U=}9+r1Hp+tz|cdj^@n@L!3lY`uHK<^EWy8ZJ|gX2X141I(QZwEytLZ=P~9z^ zP;oo(CJ4Qr1heM-zDLLUXz9sc#&ljEYiURH$=>yiUjCo6( z(f2;H@6#Ro3Uln|F!uj=PA`3rKiMDl57Rce!Jp^~f%CG;#Gj6DoX~0sFpFLxXUKtP z$JYIbvWLx}Y|1pf)Wg}`1;`)SrxRDS%&|AcyE*j|e-W{Cj)L>O^6jfNpXA9-A|5+WY(i^ zfKl4`+pX-&u~%xZB)#yAB%s+IpaTSFFq`EGyiAE{=3%F&0)>^HYq?WZuu-==c6cj4<} zzT@oDW#^6!a(#S@ylRLxa*6#FU7!tlm4SVgoGqLG&8D?ZBj23l{cDJ<9_xw9AA8?) zyZ>IsAiOrsrp2mp@!S>sG6#YKt6UoK>*eT{^6N}w{d-Z{PM82u7uzz?SbsyJ?Ttg*RKCwn$E;f-y&d@{$Eyr%0s3#V4 z_P_T#%74TCPl4r2ZDD(*&V8(xsps(-Avwc@cN{I16rVX(q>>rcZ52HrLX6 z;BDl*uNk-sGb$qrHKbO$C3qQejdp&Ex(=KguD?Vr6;}>=u zzp!J_x7gON!#VaN!#-P`mrHnS|0#6FfscMRF-{7Z5r9Ck%gba{5;<|JH>c;Gns# ze%}m4U+S)HJ`NwZcGtGHG9TXpv+zl6&%_cr@0vnij<})yQ-y+4SJW zRlh?P$(^H|@wTpn-^=h{3hXn5XJ<6{=KN zg3~Csy0BJO7p*s9GL?hOa)gp$S9$2CEFW%dHd7P{lNeIgBMk0gHyRTx?K32zO>}@ zelm}?9uto3r=HuH*3a=kdcPHX|C%0j72pO2h{qU@z_hRhcAKA`0=#Rlq5#O=myqQ&lF4d~W;Ug$9 zkbh(CvD$r%xoA3P{OIg3ZN%R=tK32xTlV*2?a&gQWvspCji{=Hc4a)jOXW`5RD6zD z7v#Qe@i!5kn9lPvoc%VPA-cGGYn#oHSH}EFeIvK2WOcO;81)l z-s!KEegZz^&ZKhYxd9s1zj#RL^K0ORQ1bcs>s812n;&(zaRQ9M6NC3@1gtAlekC6k_*#5YMX-18Q;9b_Oe-&8_(Yy$||0J*X_4> zzM!FPN)+Xd{t}J;8k`Dm-vWc`-{Vkvbo+C)1AF)xj5n#m`;6h&@cm$|m2-(b+8$!2 z!TpRfyY~ft7rC713c7b1`oR&(x;S$a9uiu&@u&0G46g9k+}s4NlEIZBdmT#uz8;(@ z-2OMn-gUxbKT`Jo4Y+=8+50+e{v5KmU&8;By+4QS9STqX9I`ic>8WLJHU5hKPxhW# z_P&enVOO&EZt&NY>>Up;`~=y%C-W4t_pE;PgS)b+IdkPlTG}8vO zhiB3A<$hDqr#qj+BvsS~m4BUQfkkwos~Asn>Q?l39sS+r59M#2=yLmqU*qO3Ajd&B zaT4NZ(#w7)7s0Bi;i&HP4N1;XvoCWsj#7YTjyWlMeccK0bG!bFka*iPgIeo~G@M z(V~izbMHK!veY&7O7>EfHTuzfaJycw{_t(~MUTf`Rr0nTRs7D0Ggi#bon5uk-aXHa zF0&q;{VhHAfW%1b|15HL^3s%{$>rw_Jy$!YU!|^%bl^LXuRBUYwJ$o&SE9$*yVX0+ zK4XQf)dHInc>3dO*JNMfb5h^>znZf5vQM@4{g$uzu{$z}1IR$X`OBTM2Q_n{NM5ng z#oRkMF?pQIhk|0gXX3OT?fFn1zB#}*;t{vM^%l;!cZ%PX4S#raefCk|(4OC8F<8PlE% zO}&4>df&ahuN~B_RpaS6TNj)~F@_1y;5ht9u7rm0tC|x7t&GE;q!GUo+S`-dVECGe zP3a8gp&37=armEz9Yyumbdkc_4!+0peH{MPtE_MCk?NZbKeZUXsq(vx^80})(t78* zSe08y8#e3RHO4z>_ry(|eHtco_>yc)F1pyjZFNlhcm-EWLyXs)v(Z;1bDwy!HfLtR zN?#PZCjK->+F=H-LI;AgkAnI&{Z}?7cQK!{gZfnCWiAz6(Oy06>EK*snU&{kRnPza zJZForPyJAyvw6VPRi3lW-C1{$!#`J^vohMVVmxw?^|j!qBL|t0H|+*>Hc6iHr}R>9 zY3MabY=vFrIa?e6a}lyPn|bf(!?N#pzEAfea1d&pl5JE ztqtFQY)Wa+QaSm@8tlYqry6#;3nVw{K=KO?%kY0bGRwcNF{YwKkFLu`wnx2HSLZ0G zeKJ>fmYl0cm893y*Hvt*t6N*}){|)>`}LSQN9C@%+_T*8mhgQh<+-oDWDy{Mke)7kbQ*~+2QCa(>gZ3N+^>rFBcFsTfU&hT=e445hZtyKy z2>*S-xz_&#+$jGrGcVo<TNt@YH%*FtW=$cuF?CHqu;|qRh20>ZCk)%WvwO z2T#@8c#$b{V>{)+Q{T=szqw=DbIR>UhUcR*IP(M7=a|szbJZwgeI^f{u|7w8!#V3v zYxKHNhAgGMbEQ2qH=pYFMhm8ney3AzI^_(QN2|KB?pz>sX;X6dsc&OOnRtk9hgsI0 z>rNNudw&>aX-{l4-d}ajgQxnxd$ftqsP_H^&fym9Eyg-3{@!9QS9Lz`z=fr!4F6R} z9z0cMku?{w1}>Qsdr%&{zRYi69z0dgHM&zCJoRq($uKSi9`V`igv+HTmrG&n<&4{c zGeGwfy>HJ%&c`JWp3?gYdbaC*Z8z!(j^3gt+e#e7r8`~dcp|rH&!Ic(CH_15{DnT^ z;?(^{gV3v&f;ZxWU)iolA6$bT(g*lNSDmOi>api8C;oeRe?97;9bH`f$P3Zu2a!*u zgnGnyzp@g$t-QOIdQsGi;m>Ms2lD>hoI4+QoOp)H0 zE#+HY@^0Tuz72Lhyl-N1y)T;BdU=i_2hCJ+wk@P=I%5KFt%8f_0U7>uJ*qC|0=NH8 zJ*K4Ot@^r3x5r|6E_Uo_2TZ-Rs`2e5)>Y;hraDFCaKd zWv>uDN~ayoFSy$AE%rh15zU{4tNn~+h~_9+nw$5)EXKYx`7IwXR^)+~1>kZ9-=kiz z`(>OttTAahIivC@=KwEJf)n#w@Y%U__9y=T8o2`sdb(TXuAOUSU(_&n-xBtMjH6MvHgbWm7m8(1 zm6AK_couuAyVz5`ZSJYwVNZ3KJ=KQTt4j{Er@FgO_xd>MtkmQ3T+r4#?5V)5+EZmN zO&*%be(EarQ;&1UgWymD*P+_ZU_bS>+D~<_PwV3-aRQ5@QQLd(IhtQ=(t+puf;Z0U z*bnD()|J!JzmMrhtj<+Cd_smis3#bA5G7QsovSPTY$tt49?5TSh~P|~7=(Ea{FrUT z4|m)#tL4{>G4+mRL3+nBo2hp+;8QNStQ)$E?h&Tq4p;3wt4e(PS3PO0tx5Qk367fw zXZQ~e&-8zB@p%7RjjpaQ)`X!^&LQ*~)Cy>i!+>9m(#u)A&qFz&)n ztpb`#KfBIX0WJOVEN4kBIQP$Qs+E0h1~Q?%8)sbHImNmAyTS+M>)rm{|C+lmpS^i3 zb2tgn*lSASs4IJxrTQ{oo$mEnw08}(w+z}# zhxQJG*TeeBa5R43HpQ@&^r(0AahBY-TJzuASKBHyCi$YjzWQ=}WG^G8)`%N%=6*h^ zS!)@o>FyI6FsWSU@P(Yg#$flj?vy`o1>R*q@37`gr>LutGpKZy=fY9LTvj7mJ97BR@{2OUV5BzPkT@HLF!62OhCUA z8gwEbip{w!W-h0gbGgKt%l{$rV5NyWP2viIYm zB7V6k6+e@)+8mAexQ4>&F^$~&r~FE|4^ZN!<$SLRACM-A-70W+vFo);K86W$4vF6Y z{Ptz~v?1Q6mm~dU;>V)(@UF#nZQX$)x(D%e%a#^g;yw89(f+}lj~%>&JUNujtBEu3 z!jAPH{8;xsPPw;h#(EFF+RcCH)$abbe|Pvxc;}h3b8JK5k}*v$M{&n--}JJ}hRpZLw98kv9>OPvm z`L~0;r?VjAZ-dC;b0AvV8lCLk%6gyIz`A&lv%)($3;&$+`>A?#p73^YK@2uJa^UDu z${soBVbAi3dQ3&kG|emJi%%S^6@9^l9F&}K(q;^8Hru1g0~4KBz?x~r-C8lXZTJOX zSKGKqOY}A}SNn$(yUUv70-yWm8-5!vJtjWz;=d7Jk#nr=!o$dv^t0T{#fR5{&dE3I zz?%0HWJft|IU0MKv3I%|dq=!IJ_nqMZ%lT@v=Fu@<{3+7kwpQ~WY}5URciNOsO4INt z?prkOQBPvNPRx_LVLP{?hUjX!CVE77e5W<=bQju-=i7pm%hqhAnrt7v?A|io&v{PW8y{T)}q0s zH=R89!SyVBO1*pqZ$iZZ=Ignje#JjEo_(Jx$6S>K<;{sJ=-e0Zj0~1p(|{3Oi~E~8 z+mc1M{ryr?M&O4d@MG)umj5eqWEyhB;nGG+y_={Pj0?y3iK8W-NG!JV{%2`(b|df6 z&*zjO!{ph8p22;T36c|{3|}YooH-5hOPvgUb5>t5THYliGjkrjaySBgFJqD~f zPiGjs;LB+DM@(XVavn6_&*we+D6wnB$@`(ahnGFS=y)H+``*$&wr=b%Q`jHP$)_)| zYvjB?$>@*%n}EMlyQUA_u^8ID%j|Dv2OMAX-m@^;yA2?S$>cW^1iS6p12>KybhU^|4`naW4<5MQT{#Ndlu^6XRE|-JuQF!wosoLhuO}tUdoJZXkhwF011Jy$N(2|SyLhl7aQwQ)3L~ zs1-TgyYu{KNZspAd`<4a*TwSwGV^_A$NLLan>J#Y3_thdiP*m`Mqe}M(qYXdG^5~z zhAi4Sr;EN^(&m2M`)6dk;N6e>yD5?WOB zMA;&Uk@ZE`-6j~sLqF7Qj=z0sSbe~IJjs@?Z*HroAwIDu2OTMaLTYPS}dlg2vX zL6%s4cERU2Z$t~Nbw1~@@Fio~O5g4N`#c}vSz`VKpAthbfH@L+{Sv*>dWSD;shs5= zG~dbCg%9ExvjwM&y#-vgGXBPZat(~7t8y})a^|m$Uu!&1MH@8YVh-P-oltWqaK*dG zQI^8~g}K20K%G@JEi~rhyUdQcC}Pe8XVSjN85_3MVEjDL1ITA7!oTELxax?E&}2@luW^|Lh2|z&|HgdppNK27LE6V?}F=X{x|po zTuCm8YVe=KxN;V9cS-;a>%8AiES(43SZ(dQk@YaB96H2F<%ZteSxyO_1!vEheETQH zGc(k9&>z~zW3Jh$j3?Auc$T~mu@?4{_vf(B6ubnih1M9lt2qjYy$QnUUT~wGv9NB>TSE!)^<|dv?K3L8Q9M~FNdQ`oBw&oSy*dk?znr;80 zL$+^__ZOM(FX?#im-m;M?P%}g9-(VD^eg=8$nQ+o zvfc_^Yn*HKXRc2AZql=Se~38}nRk5lAMtt8@p-ag^Xr^rtF170?>EFLNY0=Z?8RbR zmYj&J;d`|Fk?nCf;&0_}894~cL=VQ7WEFko3f1=X$n@4sZEMpEUG*vc$@ktznY?c4O~b^G z>`fJG(|j@U2%M#(yQn-uD(*&dyW#iUjD11uI4<$`SGmE6$theorU{?+CUQJB(6;0c zvQ5jM919(atw+7%{qec#?)a@Twg&Lh5MMnu--fMdFl#dQQpw@d3;GAo$e~NI%Zz6Z zvY3lZ<|G3@sW(*&9c$<^ZG5=+Q7@a(5HIJ1%NCU-cqO0TD$d_lvUiDQ{N-YMnwDVL zE?y2dY!?e6{Y@h+I1?-Wqr~w0Q*w*`zk`P?-BEbzQPw))sU`5!DB5k?8OfbKeSoJg zaGe2sX9DM0*vR@}Ba7r5!A{--o1zUlr#TgE2o3}n1p%}X2W_Z$vt)xVgw7giCl~r? zWc>)Hk!cCql;eW)AE1-4Q_xBGuIOZ^y`4@D+zXvV>sya?*SF@|>&7$;X5QvO4~zM` z*`SpM&e$3Tn|T0l?w}QS_t3ObW}|F;{@C&;Xb1mzi3PX?SuAsbJ=w|RcmZ}SFx#yLhcu$0-K9T)|nB{nEM#HMD#S zG<`F)eG|O)D{P{X%Fj>n+p|IZR-Ncw4L+ygWlR#$66!sO6vLV2NmuMr28RB`wot@KZM1Jf-&FSk!_ z@fTj#V)3~~Zn^&|pHFAqY3K8gb@<%<%BuoH0H4pz>zdEcy3pYBvdGbj&&wh&s(jVn ztEHT_6rX1qe7@l!_`LAOQDR>bKHmjD`U`ut>>?lEl6$(V{J{nAc^Q2lq0MsplosLh zvcf4XKbFs5`-d^6GJDS!i_dR4H#DD1e>P}Y_*`HYJ}=TcVq{K4G@ zpZg1MXvyuV0sqx4x%h0u=cA=ejS0W-c_92k-<+?*=f8%2Z-I_)hMsSNu73re--vC` z;ByZ-+cfyRP~wwVR~JcqQUIUtf_GiSCl&GDT35>l{zCEj_O{{Nxg6gja0s8bEiw35 z@F{#A2#@gj?WP=Xgy3^&<4>DA^Z6yfAIRs@uJCzayTa#J({3=IOM6G$hhjgL$Z9GlAZ;$AGeIC_IrvYOPXJ3}ycMs$~KcM$Du-=B$`)FI~eGhSW z6u4D-pS3<&m2kLv1jv4x1EMs_$vd3q?2EXX=4Sx0W z$a}XF8e(1OiT%*x5 zF3un83LnpNSY@(~PJoW#3eV)(#&Z_WcAnilYdmN0tXGz^ClGs$*cqFlW#q$13%+D> zrQ9J_&dRU_!I0_w=mJTpSa$hdKveFX~QQ2@m@%k6h*5WI84x`+p%4xQ6>P_I; z#`8p;?L1%3Gk3=qX7j99ip&bM$sC{ky4o9*q1)Y_tCtQ5F&EWewN1GL9&~7$nh*b1 zZ5RF?9d0`B8oo2mHp-bF$&GF42ZA@HR|Me318y4Znm5gqB@3ayK)Z{a&0zm(&so+3 zSVZRNK{?Ali|D+cRWun;jyQH@`->dlgpvOY?B7dg$CJIO5B2+wfNQk(s?kI|4FLRP9KfzL20|?BifZQSm)*6 z-f#_T$JOfG{GZ=ryGM?TeLn^NxM{Xb=G?}7Wk3Tu^EIC5FrKq`wpV%{72P(Xa#}HT zQxab383aACpO|KYUhs2q#$>V%YL(KS=zayPH_5If?^o!#3U2r%3cji6hd%&3l+8X^ z$~F&Q;u9Z8;Ty-f&J~`B%PO_-h)T~z)nm2M`C~m1<9QAPo=Yoroqi)SdA9MK#WQe4 zxOvui&fr>K^zjh*cYWP9W!EFb!a;X32ad1XE-Zt_ zUgw?bn-02riFez1SAD$W-NzO!LT_%yFa5~+8q;d5V;>`vGRe8~BzZg% z@fnmeUShvDe@h$H0r@9-lS|?|-ibZR!MS^Hy>~G=f%ZxctQK-l1eX2I=GZIcA0YRH z!*1rD5dX#42bZ^!dm^py)|PT~5SM*w`y|Xmf?S2aZhwCc@8#?#hkKHp zdOSXh}8KF^s z7O_1Q^x@1Mj0wZNJPP~Wgk;vCLE>W@Q$G=2@(;qdqepaog5JL>zMwyJ-f9^@uAq)qg%nZ9t*YJA4n^6-U=$;}(}5#N7sUO01B^OCB07ku@z z&r1vQa`6Asyi_qSRUzhO@{gRC7UreubmrwxGk(T-iPrpTUXVY1m=~FU;TcB|&+NkI z)8d)8Z9j@{K1rOemu~q1 z9@)=#S^s4}lp{XPfi@v{BH{}}FTDdfpM>2&;-o^^s3liapl-@KYREh0@5`gco%3q` z&=mq@UUR_w9Xj3lOB9)R>(3(d-U=~)e`Wqc!O}5*{}|)Je(Yc##Yemv8=kiC`7yOZ zpUA>KmK+Rg86n@^;{6wedW*=w%8N4m_+i&uJlIf}KSt!EB^!5%UVaWTF;Qeuf91RAbE2Ee`dN)m zW}TDY5@zUMpE38raRGAg75708s>2^|;;Kk|5}kDu@Q;gn*tx>-@U9gJ#78)=J2p9t zIq$}r;7D`(mzZ-NOHL$N>z+{a{zE?6QN^2_J?hUF-BB~vxs80nb*KMbf$HzZ_YH`uTjt8jIP&f zM2>dE35eccLnjd2iyyV{?59&jhQ_O$HUItt^VY!kHoF~ubfBL*(a)oh3DN8oZM5m& z3SayT{_ggR-u4?}BIS(u;kgpu$y|$nrJA#>K4Np8Ci`o6RcPn#jmTThSen-GY)dG4 zdLJ@?yl7%u_C)BFVrTaB#12nhr=0oS{7vIzFTaEDJNYht%5HSph{~M#;r=4VlrulV zzaS;jzhJ(@pEKXdZxp}4IqwQ!o(Sv{unTc_l-;h@`=WdGQpukrc#-|i_6>&4d6~$c ztk5wOQntT?^8gor$GNNFPJA=ackcTQdM7mPnU4MX9P}@Ij!I}p<=;Y2aH2n^>5lsC zDcX_pUljRr=EwRMjL=#aWb$__Hm=F&Y1CgZndhtc`#0WYW{k$aYs3=xMSRwH^DP6TXIQ*`KA&a)l5UN7&m*<0Pqo+VUFf!ZHZem^u)$XtbS zu6pI|*o*LK32ej3{z}gLt#XSfn?*Ufk0NjnDr<>~6RO_cqT#oa)1s9sS_|}91|HU% z$xBk)qsMW!*i1}CPNp_Gce32)({XMtHh^XHaYKkUG~ijj(AZNS5=}#LBWE)*20O%Ri?K{3LobFEWU>5{=k;@K;M)!q--;y384eLP}oi#0eO4*m+D#8E3nF z#mteiSk+0W=!yMDzF9s&ufku_-*)=_hO>F`&&uhdUvf`&>i4&1nxD;|N+rmmhwawX#ihUHg zuGO^BGl)lCKul~6=WKGWc|JTN@6ips<9M&bZ|Ct&&N-|)4n3~k!MZDaBx98C=kWbS zHJ01eSboD;-ZaPZ7sm1#V|h8|@{-Re^O-ZU{xin1Qj5&<(EeW-%RjY?Y}9JzdwP1usDD>{R`AF9RG512c&O7L%euVzd#M+4?gtGZnet9uFj zzX<+c)FSF$0{=DOe;@T_ezUu2o73T!L7cZ2$7-9!781qW)}u4!p)=h@yTf`g_r#SX z_cZ2K_B5GuXH|Wy+v;wk?HRN^Lkp|BjkXJDdxjQQoHBEU!pCjkW08rEN5IF6;N$j~ zOG{p)%-N3c`WL~+R4qKO8oBle_;^7J>x7T-;A1@a5S_J9x0i%!+nQT}tCBs%c4BWE z!NF+eMxKW-Px7uz;;5Lnvv{AWV4SXCoB@n8Oc-wi#znw5K4v_06jwaJ5mvtl7(dd( z)STT0jD?!L6O38F_%VB{!@#(o_-dK6rSxmT!a6f$82G&koNa(ly94KW;55$Gjcaw# zo#yvg5jTJD3Y$%@%K(Q8*Wgj`{XT7NU=MZG++kH#d#j*j(dA{l4!&75N$d`H%Z8(0 zMxbX#qHhv+YsL!6epPgpULNEIch2>(_o|n?#20?QVBhNH*~fOV4kXc@*qS7Vr072D z(K((1-ebtrGw{<+Vhs?Pt@9hla~ZjN(W5KUxl_D>+@8WK*r>EMQ2s!N7*VHQt^L_;A|^0qcUI4w(Oae z*xbDZoNtw)r+#EhJSg%*_8&rj_bJ-D9{l_gJWT;#*MYamoWn@`Vz}C~3H?bPy@phL zdrY5}4+}&4M9Y1kl6kM9zrdu>{5)U-5o{UW%S~&e_vUkc@GiWSQfsZa&_#Eb@}1rDi_8sT=#Yc_pzd{kJF-bz_#o*KE`_iH;ve`|i1UuW1hqek~8VCvC`kr>^;(PT}Rw zdZMhK=NzDp`FAhB&H}j~FMqOplfS?_#xA=-Pl&u;PdJA^(T@Z5T#2ic^632=HCKdJ z4>#Jd?)uwAEK)MLJmg+Jm)8G%`gqssBbPeq1(9Btj!jF?A1m+GyP6R1bVsLhf%`zA z2M24W=;zbGxhGTeKI1fXR;Q&a8~U8YLMy&tZgun9pOx;-889^TL+ut=?M8QQw=%F@ z^IN&u?(oxSH;;CwvDX&ddQyx%gxp0WG^hvlwcPA0xct(f@fAzsJ+nm@Kbbu{*QGR8#yueQc` zqvn_OiEm?r`~A@B*TkoTdTN}WGt6;z$N}rTOO2I&o(mqU*@pNE3yv0=V-@?BrK89` z_b}gs;iw9NPYAkMt_~&h0BfDRZejWuLO{%UHw)l5@G?7bQ8t zBu}HN*AijYOK#5%xu3G(locGep~J{p!|&b`^K|@Nja;J8o>BjT$dl?z4z6$1I6CTg z<%=Nn+|@m4OVHbeekmEyJ{IeDS>ydNdFGEarX*7 z22)l=p{uedF1KmigJRn&Yx@JlT+09X$}c9YvH*V<@z>JCU#qZiOc85a({=cI7cLru z&uUB4FwVq3(PrbPRw=m+Te0tN<=pkyaPA*qPvcqKaev5};=h4D_Z0MdT1dHrFV9GD+!uugd?}Tf_(P8)TsDuSAD_mi~>q0D7T2Csiy3w^jQ_s`H2KYexCE z7xrn{{?`%Qhhg(JLtCy@ns+rgY2+@1+2E&%xB*$8WE?V{GU&c3&)iSV?l=#7mj0Wj zn(t?HykE}yBK%j*)Xyv)&F|Ll+O{+-&f**dJ$oQ~mlXDK*RhYAh>pBCgZJ3`7iaQ| zKE7DaOSH-p^KN9$Z(#1PhX#HLElfejzK;Aq5sEitoD1Q-Om`~0mZ>?3b|R zPWc-B-hmF=%j@16ZxbIOy<$9l8gpvs`=fcz2L{<2$@!JotQ9U?mw=1jYVYtD`hQx{ zf$X2+E25zR@jFfCo$TG~i6!|1XAIycr2Z|uw3$APwUqqY)H_qg4z6DYKLVo$jJ8X$ zEAj2sd$k178!Z1!mDBdzOZ``nS4(vM&$%Ze{}F>%VzH-?AA0#0!*##h3!Q$UyY=8e zn_sPoWy#)#RLwh!^B~z5SZiV^TuR*6sPW`r;C>uO+x&gRW*re+PPb(rYevr&dxgS_ zM{?bY-2(i97unAqW{+s#$H3_)JeRS5lDIJ`x0EqHb*tuo_IlmFvS(-<$sJZF=5ddD zp)PS}Td~6{KRze6Iftwh*f_9%a37YoCw=c=Z~4GW)IH7R1IPJ`pj}5nigyCIYZ|Dl zys72Ld&#L0#aht>4phA(Mm<$tH_9JagYAUV#UQ1qzI`gG1?T-uT8@UqysmM}N{-f1%z60V< znHHX1;y97H&w)>yZ`L*+Lg6g_^X$U5&3f8e36YEdcAedB>UlVfT&Y7L{hJ7-$Z)CQ^y?iS4VF&gJ$FufDAx8)EhwpsrApXY5Ssw6; zjeRhG(P6yjZyf8f*ju6)>tOz(IcFHe-#Fqx%a~Uu=Pz;w6BUCzyZJy|Oc)3%H zDjCF|#E~wfoCAE>bjQdTaFj-l^(*)rtZQ4LpRGmAelOg}XC(fvtR>tJC-*HSRWtFnkT-l zf#u%kp0$&FT}|Do7GKp+_pHq!)B@%C^>)q8<5|{5xsTLt4(rmQSwJ{a>*@irq7fx%nR2 z&IVuBTDG6HzD#KJ95=3`TCtWp^SIiOOY3Ov6Jv|bry?{-4J}S zV(d}%Gxex?a-G#p^Z%K`h~ih`7=6;b$%#&FgkzFB@%Z*V)0#K0p{PuS;F zOKc{0XbJC6M0ffSe8(UQq+envRDW7-UOP;3M)TPBZCfP&g4UIxnW${p1XOO(K3?{zSHpwG7hzG*=xKZaf)bM4H1K=Hc{zc<@QRXmsCTp@n44%(Rs z&hT|9zL~aiDa-#QXV4Zo=~(}bG5r}`LC$JUpXV=!E-r&UE`?6ULoZqAtlZULH*%)A z&SnCFgMIiMDzNXF^fQ zZvWc#OW)1ww*Khab#EO#IP1_+bgjB|Up)5i(%qFuhfI3==;x1pel&*pJ$KUM?^g1z z^7GwC&wb*HqZ1{sDf^0a{jyhP(Z*v#Hypik>F%SIOAj4Amv0XK$jx4I=Ucn$^2n_& z{&-h1pV{O_uh_P`ZXNkeqxqhTk6RA8)=S8ZN}k}7LF8qZ_mlX`p{(Rm{XdzPy({@6 zYfyZd+wWl=6h4%^Cn6)0xCeZ%(qWDLZIPOHa)30-UEuhga2I$m|4%&@S|*5oEbUg) z?gzvY3s3%$wypaDr0sInx4^cIxTeI4zdo#&KFXYilAQ|n5I!tpiHZ6tJ}kqi_ut{e zlGp{FLfJ`W?~uS6)?ZIJ=!9oItf>y>;A>>0=nhYl^H$>RWj;OzuWiIx_0x5`^paXAAab7}9Tpo9I2b*K-z z5ckdQ?H#AJqBm@EoWJtJV%A2{Z_X$v_T49Y!4n;O!PU$KeVlZ5-S!~&*HLdlisqL) zPA>w-a*my5<`A~-acx4sI1^bpgE=zxi;2qiawE_A_<`Y9we|Z#!^Y6Wp6pr1_DjyF zFQ9FQtkwLVh7F{T-lsU}feWhS-X-z5xsmm`6+NdcKp&EE_hH;~8FyNcjY#0YwnmKh ztyN8&X^TED=aPa08#s_Xv-q|a((fYvR^U6^$a>OXo?A9`x6aK^y++CWO8X+a1a`~! z`7FjOd{Yjuh`m?#_vze^G?RVIL~`S2lC!j!oCp(>&uPz-e_@6?vs3438ta?L>%coA z9?z1ThaG!U@I{{5i@asr@y!2w96b`ePS#9+s@kV6bF4}=_sgz%*hs$MGpm}20d_Qwq`$F#xeJ#4_QrItvEMG= zhu&k;mZfhxkA1t@fARpY?B|+Z*8Jr=wY|CWKXci z3~J0j9}G7K!%*1Sj?CPR_M7)5>Fh%SY}3n<{~jf_WpcTk-EN!S!8oI!Whb^&(}uk) zx!Nq}>L|CLa#J!o+pwoWAIV=wu|67i7g_f0_3jXPE0GOCcPfV*`n}kk7PJ`WH%;FCr^CP0Ck1OY`QT z!>M=G*q_#D-myG8h(9fh(1_RXU0*&~_y4=HEp_HeOJ8(LE`aLZ-s!ZPSv|;`!8uqm za!b9>?qj@n4fg60+=0b=k-uGyUDhtKmx-Pk^+Lu;x?>D7SoFtA^vB2070*R~`~v+^ z^ueL%kL!829{q8s(jU9k9U|6YHoD^z=#E#TJ8p9)C z8+q*;IRB|b-xPiFi*0Y!4W!nW z+EO|9DKGi>*P8kD?=st$ocnSY#UP$Tm9z5o&)io3q~z#NFL!r z*PHp+cUk$^IYX#rPoLAhBTlE5dm5}boo>3mcNjD(aT{N%xQ)B<7jq4r!y0zJnOoll zt!Vg*Y2Dl<8gYy@eYE;yY=z^rM3r0Lg}tSR)SYn37O%G`8o~*HmtOt>Eut^n(92CLcSyR`8E8 z`P|ix9>xE^8{_00Vo6`ERWJDc7*7=ExCPIT$tCZblX%1&Y)&7Ux!nuMA$ww8Xu4H# zTzW@t{!sCV5-0K&xRv-H>AN8-e@tUmf$Fm%YvKPU@7?30tgik4=b0pACgExl?ve|( zO~AWAh+;EIyoBJbh?myWLjrA0z)MxE_mTwL2BX%&C>5;)ZI79;wA4a1y&M9xHi}w> zR*&{V0&SfTZ-k2pg898a`+1&ZG6eLTZ+l*^^ZNZUubFwCeOY_0wbx#2?X}mQ_0n*w z=NfF!d)zaVvO#8JpOG7=hqL*i7nnExaj9F)o@>!#2F8w(gzgM`;&XP<_sOLI>^dv7 zWwSXeGyrz=rc&ru=d*f#uQd`q?52OW8@|Qw7Ho!{3H|p1?Bjlz8n?_~seeW3O);l1`}>bL{L(?MH|jh7zT&fJZSXIXKPoqj2KwQuBO!*ja$8U4S@r<>oo=(Z`e zJ9X^++n)YGbJya$tzGqh|DX&1#fzzT+`Dbg3qR_bvv^b2)L(wo<>wjeOGNt@L3{No z;l!&yXX*pn=G^gvuC1?c?@IlAx2eCiuFf~Nbj{h{Hfa6${r1@X{VLX@sTBoB?C%#r$LyEJhf4Tf z7#$z_8s(mED+ry>T}tu$`};pHJt5{EOZ2JOdo#7a|KIAT{7kdIKUiOH#rJijK6J$X zz8MSq`(I%UcekDCjHUdnBlh>dUN~@n-_+|K@76o3PlwX}{xHgF9vy3c-_)Nr;fVU$ z-#=`|(O*9odEmElyTvbl_|0$Uncv!%8tlGU$sO5Wt=OU3-_1hzQ69lHd_TyU5BW-@ z;~T$*Y$o{k4RZ&B<1dLXV|*%9X?FasT`}XW%{;kHZ!r0^QT-Dx`|A6Gm zdHBrE#bMj}4MEx$84+Nc8`+8CCcdw%z zC&v$WM=T^BmH$KjO}3Y_&uB4L*=Y}9zsB0@-X0zM=wbD7E`10#3!mY4rpq3~9v!J1 zm96=e)~osdc7dIK>l{0MC7<|rAMoy{Jb#ssvD*@Z#CG{3S?KO68|A&e=Q!WZh7YrX z1=ttO%AZqc1v=Z(>c+eLnI~OZXD5nB&OX@oxz*-*7Iui`pFAlmD@$iI=9RXunp=7I zQ#V@n=O&fsW=&GvT;8n#Z~l?%j`Q(67<8UnW3wj(E1lngtYEWsLh#pxwsphoGaj6q zxA&0f@))%5Mx-jCM2`<%X0K4XVFnBVbzrx2GKTdx$D zO}(%7*PA^E%=7I)=T7i7hjFCl*?F|vJYY;G27BHCo_E2=HTZQ`+rtb#Oxqn!+o^(y z`tdLgg|NB!knTv@v^-aT!>^xjoGKwTI`I zz6{Q12Akgn&pSASTSQyK;nVjxTRH;U{XP2nfWElbtbT_*va{I3;k!;>Bj_t{u)dP4 z8N;%i`ahv=FK{+N9|LEzQ+F732cx}o`;qI53Pzn8>>0Jm>b6e^9zG4a7Y?^>*fVDJ z_?+R1!$++vDzGYp%_Du*amDirM-(?T4lhZxMpR6E;$*wXI%NIOmzT0OA#do0#JmwF zkq6*3>(`@Rv)xy(jM<|NfKT9lw_H?`Loa*9Sl0{L0A1@A7-?ANknBr`(rd<$L^i zBDnH+@gytnapBU#MITRH*pG`U-5DM?4>rP6%gNoe2%Mx$Ka!`Gi&tEl#_}3b+-8i{ zSoeFP`^|jS{9?|U?~&-(mbLKl=-c))rDr_YmHYAAUABGo7q2^Cv(M3;Jv}bg7nSn)f%lch$21yU#4a3M=Lu#>=XrJE zzhy~Jc=h?}&r9c77q9a?H!Du(DfvYI)T{oVPV4heu^070tbb~`kGK-oKgD|as{d2? zr~WWgF!cMUB)^5fOv~ zJf>I~#k|bcZ*x|tKknuGW^Da&FTk>Ajn)*#m1dy@0`LC9 zLodw>Oz10^ z8gmexa9~_}Jl-6i9dYO2&%sT`*7@iUb--80*epAvUOLV^betyItxFlaUD>4$zI1La ztq)(*TzowQuHHfZh%XOZk%&H;gihtebp_Z*1n=vmw-5s{Z%S4B%=t<7Y1i=E&-mN< zu9ZH`ov(SoHKl5&dCyt1dDy+LTHW~VICHWyAt`eD&EyhbFB*Du!qf9osG~jiN-MG6 z1}9T(yI%LNcH&=dt0y+@yWH_bK5xTwEq)UhcMkEg(Wippy<<4TZu>hQr0;!C7KJs2 zOgp1f`t5RXlLc(D#AdTLzs7TK zk9wtYHqic)+~X%XvJZUdTePy#H+7zdajj3W?D|U0!E2|70gDW}mHx^w>$tVFx0ZI+jvGEj`CvBEPLMhqH<25J@72SC zH|Pwl=zlMFwhgViX66M4lk*iz>ivJIXGA__w@S9!?%v1E`A5o(-*q`ML$V3m@73%( zPYgZ5-1x_t+>?04ui-_-Pm%X>nHBoqk|WpQ^ZNgz94XfQp#K9olH=mblOulQmE=V( zyn=k_lO6KS`H>%)$d7B8r<(KP4=>Kx$cqO~bo~0k;l{838S=lx>M7ic+rJKRZ+6=Z z?xNOwOLFF$*H)A@BETJRiVxdFUjgQ0-nd|ovER4{r5w1tm}S%NB_}!kTEjxh7xCAd zuywa2u%8wRWEFlbNh zfxIuA9*Pp*%bw@DvCB>iO^3g`x%+p2FZTez*U_`Ap7J5stIO=rk7n2nxuos;3Di;Cog}{9Q@Xui! zWyIrfhvAllP|(VJy%oQ<_UKiA9CMU>p?ePEcb`Q&vG26rex&}dvcpCfsTdZD-ZUiK z3;gm+#KNDCwDzsBNtFF0k+wzNk z+?99DOI_g$-tMyO*$1JO^uQ1NYlgZ0oguD&2Ydjd#|!K6kC#JN-C<<(>r+n-eU*AY zY#XMvDkQlopSX0NQSzmK=bZ3Y_$Ic%yq^Rv4bE$_L;JvmaMC@QT*TnL13YxZ^!149 zqvy6y#s~f`aTttAZO`A+yK)h0fS%VdXT0ZM0msr0bD&Kw`k^2FkUMo}`qBGx(GQ!p zXU;69oM<5(5uJx}LofBCBYNeGE{Tp9rmU&6D|6-o>L{i?7d_FBo;aW^dZNnq)uX(r zw>NWUE%nx!LanCAY-lcSw;`9#AkLh`$+qt z=aPRf3M-FPl=Ii}Io!UpIIR2+OL%t6TgBl&`o?Un;kTJ@XnYWL&Or9*|`@9Ez6`$Ci zc66H23FAB4&;`{`yDy_&`Kor%S5ulb?@iyBdi7)3v$KuA}%pvFe{j?kHh8}(k z#zgej34FJpqdopX@5+4WB!6o*-w*OXix?)gu{6QnvxPE#Yy8$s+uk#o-?zMdJ7=B7 z)Ca1$!-)CUNt;2=nHO`;Jd1PYIh-?JOj+s3ONr@h2Om?x=ZTiTez{FP3f?V%j>5C_ zyU0IoUy)60bB8abzO&W}-CJXY-bHULr_BX1XV*z2Y}(vNjxg!RQ!>CsRrbtmknP-pNx*I;uXYS9T&s1nw56yL^rXG0d z|FuGUOs5Ubq_gEfhog<*#1FLBnT{Tx(Psngy`23`+7myo>FAwDz9qNJ=)N+$DKo$C z`Ix@vf8+U9o*R3>oI7&*KMp>S9QwX=h-E(RukOz?qO#G+^3ltF^+lE!V-ta^HlEEa zm2bky>*3|{=q-^BlFZueXAKXqmZz|$D_`DF&H^To3)jiDIT4=I8m>6G3E&`r4wk~) z-~AMO2E=D&lm`oN7NyH)F8{h>G?bIR&c`{NiZesHkEjwKN?_B7P$~60CMSfYPn4pw z_$y8hMTz;2vQ|d150&eW^Mc`>TgZ(x&2Fd;SosTlW4CI}u3_%A@%})*-JtrM#m)ba zHT){R6`Q=nH__OUXYo$CfmSDs?&KW)yv@FeTWfq1>*vq1_EbWDm8-^P@iU&h%F3CA zdE8;ibL2%xHn8T1?3Ez;$LnXZPT-T3&nnHHSicln-U(mkVEfc9x@l%6bvI$(2rmZ{ zCPp}unZJ87c^UaXyMpr$uvh> z0Yd_?47B~F-#%CS-ECKP{G@AU{_nfGpWoXx8Xc<@ejRhy*hia*lPcg2nlph5+^(3u zWn=ec%>SnG!RBa!e@^tXt*h1`laYU&wb|(4&@w{ZdE@-`tG?OMTF8EHHZ;ynP??O* zMt;}%?&uEKOIKvFhm@5UJXiq@rGIMPK6@27WbXchIVZh)2KzzYyTSTg2F>4tu7}+E z<4paD5wE{Byf-!pwt=4yeg1T^cn+WYnBXIGuej-{8|8b!m(-kTk69c=H?CaZD7IrB%IEC@$uZX5^>xUYv?8l#6!5^)=3b%Lan_7c zoP=^64BQVqcy%#+!Jg+9J9Nucb~@+FdLBgX4c0~*Yv6idUF{pQSa_8TngsnTH#_&h zZ9?{!`XmtmW%|RyL7rqc+;J;jcQ{i^5NSd(6{AkQW`P)y;W#o%v{9 z(Tw^+dqQWbXwKQGr-%viaPeyw20uCPHJ&&4ex}W1+o$vMm zeA)tR;{OcdU%&Brc!oLf!Z*i0s(FybIW;p6@G9zU`P_||KUweYLwlCQX&c%~~DcRBely%?UEH`?jmZ}lZvqH-6yGMptt>e{JGH1VQzdfs>_B-Uy zIEUYL{9ev)&VJX{@_QM-IsaXIH^0BdZ_a?%R?o_X@1(O=)Djm)+g^@bFE_3Scc$By z@1N9{#(o`RKiWCO9M0T(@ucFl+n`m!mh_P9veshq58UVM6Yu2=oHt+Phb`k=x;J0* z;ltJ;!4=-VNGW=A>*t-iMd-b6px6CZ*Zl6+xg$&K4KjeX#u{6H6EYRquzoW#RnLBI z@}rH3$Zw;Z6VLF!9X>gL+^s|xAotw724Yzif3N%UUii3tyZi)_FNLf#@?{G4!uy=N zCdIotJ1xEbXy;#*)1lkr#Vqc@qpalFdGKVGYqQ3~d2bw?b>@FsBn@1PXKyiM93A;B zv1ppF!+8Fy>&1i0b3E|Oo$SE2r6%U~)$H=LP+N%=I{tLpK{sh3rr-?D)mNH5G579C z{7-&x)2rNa_(UprmtCG7lHX-IeF)}CmmU`LGtG=&>q38wqZgOEH$#J_KDszQ zm^Ch4o-lMtk6hx8&!fu){MVU8d}KZGWzUT(Tg3mp=&HiQR&Z)`Y{vKp7dOxI>}KE| zF#oHIS$ENWt9=FV)eZOy=BeIiT|fP0;8QZ+oBK0v7$^tS*9C5wQ~3VJ6%Jmmq92Pn zzhvr>d!)pAaF<(V_ag9F);>D4oA?|Igx8)1j?Sr8*yx8ROw<18x`n4sclJliM~6Pc zSwOQ#I%}GdahoZBH=paL58NY-ja9ke96Nm&X9i9*=LY&?+W#;4Gs4hs5Pyc@;cHJ5 zJ)FEe+bNd?K0F`o5&CeVBRjO_#Ot-9`KRU4r%XOUa(KD2Bet!^cY4Qkpl?XVW#iwK z%*hFeC(|R3#MOJq{LhGF!I%FkSnrJMGZxmHj{)m-arG9tuo;t5@j4`XaD;=WeQy>#_bk$rI+;1KHKnHP0Scc*!))vvtpxj|u&K za(^zO-@s1`u~%dH&(sS}H*?U(zIf?0JAJToiQ*~A0Nqupbr(O%dLsi}nVE&GS%Pl9 z+KpY0KSwD&36n8u}Yvx11`!KNeufG|C ztiKtN*G>QAQE=zbHva4USDoVjhsj+ZEmF)l>*Q0fNDTQmZ8Z7JwZ7|_wcLsE@MOzZk?WEx zvG4rkwY50YB07~@)^v;YPrqYxx>PD>i9P;kF`4VKgX7^N^ucTKiO`1U<54?a-`=y& zQ09|jDTS-NjMb~|0bkNn6jQl|{1$nKZ+BuUcisL>PFzf7fOf^7t?=h^aOWou)r+ar z*&8pWa_2%Tln)-F|IK*HJ=fq{54!7Y&{=1T;QiQjSaiD`dc>}^tf3|jUojn=y;>hz zUc8tMoT?{%-v^vUE?vE~+GY<&y4_iwPMpHAd=~yhOXrsM#O4Xf1BV3<)vaoM&1#tT z_U+qqdvABntAfK?H(!qO5@B1cD{d4Wl}9K&vVrmbft)T{13qM)WCPb&;L{qQyYxo$ zzZ~6FXQ+Dcv-J|cs&<;uedUY$2JOARU_wawCccD!O!<2``&6HKmle*u%npyqwwlN1 zS=?z&y#zZ{h;Lc?R}|f6zVgZHd@3?bzRe}<=O~|N^o?LR6QB1B=*eTio15?VmHc9N&}Q>hX`Yng_Grym$)rdjNU#dQ5vSslCI!8!XxuEx!s3S+1V!!A}m5 zbn$@3E<4Vgzhr#U&B=+e-iB@tk02v~`#IW_&Q@|AbE)Lmep2cFn(zLa#L!H5!Eev( zR=$|#AsO{6dH!K>a%dIb@?-7Wys;aDK(Y`SIj+;kK8GJ%JdE6)2t0o9_Ft~uz6Kfd zIy&WJ`1^~nmvt6fw!k;>4SUaPpkZmJ+Dh*XVl%4UD0d|^(B5d;i}!go(q;o~Qh!f2 zZT?w0Gj#iqHofQK3+2APnK`WaE&i(w~$yfyA{*PC;)`4r`CIj4zh8}gkZw&CYc=*}i;>T~sn;!Xziyu7~eq_g~ zy;5`nopZ%!%{g#ut#4S-+MOfpwa)XIp~Q^f+P{tQtsN3*Ir|?IbQa87`_)yxwF_6+ zYZq`fZ0SzxHs!!8$9|XXz7!weC;4y9oNJj7Iz5ihTeQy#&pwQtu5g!v+<)Z%|Dw&kuDE&#|!W8k;+K@)0)oI)Baau8weo%}qUWo-VUOSJH>% zU*!jlOxPOw5Z$R?W%71c_be<79{+Ud-59OyIjD06%LjO-^8?f zxjz3d*|W23>&yIJ}=3zSer;XqVWA`?W>-sBS$Jy^ss* zL|{FZJ$d*4f<1Yi3ya4qi}-&mdvZZs`G4oXXHP~oSGQRB#QWni^Cnuy6>?@}6FePj z$5GClWy41vS&n~9x#aV(&8o;BTa8|9wag$sPO-X$hWCjzE-1urSB@-Hdlq|1tSKL4 za|TswW)wTWmGw;eVIH!l>Iv=%s|^+`t>qqB_K&L28B(kdRw(XjGr41l?Fiv3=-Ex( z+3a`F=AH`LHn3DAhw#}wP*!mQx$Jhd)&tMtq7fmLS!Ca{YBqLhFdPiOKupwZbQaor zE$`hsR|Kp;eO8ebnw_B-JI;audnNs1Z@gAXU%9|l2|S7;sG|Km+Mhy>xvU=s!}A%N za5RPXh<|yljP_E9-w4uPj@zEcZ^E78mONPR0lqTeTTGjaU6=*yY`cEdiL^f(eXbNz-(cY+1!1YYgZ#T*#ABUe~uk| zrXykbs@?b!0*7|D&L=jaotUjG=x6tim|@#vXXIJ_d*a_`qdTduKVR+aLBz`6z@Z)f zD$AP27`N%3J7mVXyiLTGB5NfhbY{~$`(aVfgS@*bXIQB4T}x|B0-9h@szgDqw17hPB|+h|}UPPuDJvG4!O)|OE(;|i7{=G^^hd)-{O6_6#S$|qt zj{^99HE&Y!O$^14DSfrnzIW9WBm} zWI^y9T`=y^iOA*E$;f>0n}Hp@5q@j~mtlA|jGa6?x%=5EtWm;uBnIE2b$~VE;2Y$5 z!Y`D|Z>=wW){7>ezutmI%6Y9aR!z&SUe?97gqm3yM67<>OhW3VwAkc}5elHX3IUJ_F5%=wvHoWm}=o;4^p`yvFYR4VKT*@nsI& zBaAPncyzPoaQ{FBw9RhI4Bd38Bj;uj+w4b%HMw_9%a;-hM*?tYUTuX3Zik-#uXcy- z!Z3CR{mbt7Ds77X7VBdMdrz@8h@LkQ%OU)TekM=vr<}`els9uZ;plT&F&)y2BqRJq z!2)Dhex3GfW9~-XgiOe04rLx%(^^QbrgrcXyLaLJe&cJ-J|^&o{>E?Zhg|ek%@5=f zx>EFTa(*uDux1qKoY%if?*HNMN4H5{{`=A8{_}D5HkjQ>lxMgP>^K&8K5|8(eIXktM8jm#BvG|6K1 zSg1w# z(w5OX4Smr&tC$DU3;pPwo?fRiv+d=pa^b}_v?qNLo|q9k_x-F-n){mH$C}%MNAf~2 z7#(1EWxb91mUyuf_*#`8SCkEAtz z&KNplFZ9_N$D(r!FvtB15cBIC%q3Ii#U@f3_Q-WSegd0U1I;o()g}p(D;%V8ehn7)oXF*cvj!Hv!ADT zvw0>RM875PrVuAP^;_TG_Vi!Zbah_$m#(d^z2BvL7m-`*w%s%Br(ILN^xLkK?{{}) z*<}aY$YCHGPIWyw68qbnJ4)^hg|0c(itGxlu=PmfEZPtgc zkbA-CM&udPIaHlZtp>NB5EF#VqI?NC;PY5(2gU?F;IH`X(fpIt`>gR#p6b%5@}KOV zob$JVYsC5dKbn6s7*}={|2_ZY^XQbGf0D#<#!lMAv#$ep>^lB`$(Oid5MN>udjz88 zf83LH2*hPjEc@hR0hyRmeW(1Nk5dUtfY;cjd(Thq%lAkHmfIZtl^{ zY4q6@W8r7%udgq#%pKt^=r66@?@b)%`j*=H-NxobCq;h<`R!Yb&M2Eex>_r32iCAg zU~l?rzQ_7C(B3R3mi3igj=!mne0QzbYSr`|dvEshycdi{wzFm_{;`_)#~R`ub=IJc zb0wPl@q3Qmeq*WjQP$)Rx-W0=*OtrNu|jvC6UDC?|17`ZLySSX>R>kS{kx8?`+O&^ z-ZJw)BQhLbj^$(N+S%Aj*|QQuqU!=^wq`1^rTl*Ogn=j^FndKiU-!b(0QwW#=O|@700LRowILGu*2#TT%mth zq5mMKw$^gbz8Bu(&x-h2cl|XT-GQXwiVp0LLhL4Nb?L%;pbf;f)OT zcmtOyjscm)?~YWfCyhg4x{GVVU-9vpIEf`j*3`JNM(f$--?bV(Ft%y`ISEe=1d#() zFfUZ8@z7ol^QAZ|DRd6c?{3TswNECO7;R{+e}^)LALu6lTz=d2qq#Py`Z|lgG^eM+ zgKw$Lgk$IT*4@c{ivzG@%(m&6p@njGx9kM=W@mFLi!H-qPC#5q(F7t+BQ zsU~wKWD*qk@|W7v_u@x)v zOL#Kg_VCU)y6?9dWIL)%6KA5lGDSVkI?zU$_bKx{WinHpyIL$@*xze~wblravRTT} zElb5)&_{Ky#m4_6KZ9B4DZ9Y+2mB5>{tf6MSQIA`OTSGn{k%Fa0+agw1$C;ZV<8)2 zV`~3Q9-D6RQjFm{R^NN-#Q6No{z%SF-E~P0V&EUk?-u$lz3#rbO&9$5M*m2wITwEO zj|zk=-IvR~Xmylj-i9kV2jRsrNxv+HAB)MmskvK2na36mjAIt;7H34K{Ml+);w$c6 z>hpE)W8IVun)!yt%u#}DVh zHt7`Ii`kz`T0T3)mMA7mu}*61PqZanLC?!;;fpDpci3!&hEIh@v0-9qQRC7=eI%da zz%`|*5MRdGB_kVitOCX2|4-X{ zf%bfZwC5*ZxgQ$Kc8ndD=8ANmso*RAU4=X5<);mvceWfG+R8dTv;dsQuKU9o;El1# z#+KeyNq?2K!2$c`p1-<0r=M0s={vq$A5CM*T@4%_-o$$jJ=4s6_2Q9z(Ba8A{!^Ly zKUoc0dx&T4IRlt9hpJz9=90$?Ub~mKADJIl<-Kwj!87XXP5O%EsVaC!&$siu78v4n zGtC1Z=g5jVM|SU-4xVD^;PIFGdIbJja(}RUnsnREmdR6}aay>$6rROrdXMx|m9O&^ zh5h&|B#SQNuBX*Io%^nY_t>`ET{^1mSUdK+rwX3~<5Yz&(J-ntEoLpLfS%$LwNZ1b z)u1!mrNTAkC7-=|v2|j{^cBV=nls-Fo|c2BnS_W3wodnM0- zVJ6StUSKzDa{R3W?>EcFJ{ zy*0+P#$dP<{ve)hTOM+My6xX8-9RySuR~+q*T((;el}#lqHiE~96L*U0pg)@&M$op ze%OzktAp2a!MDx*Yr>J_^5M8U=58=}I`ZB*Z|{v{|4|HI}v!<>)2zVAGFej-@et^M{r?PT-I5@dH`7O1=o_1l5auggW&d8 z50q~;t~|A{FJC_inHvAzor`*Z20UowSPT!D_Q?x_t~yV1ASgXBM#kQ8`u|ME{tQ?? zjf@>`=0*RyDSC@8n~lDb9(fd8mVt*n@BrS|=i>9vO|p9G_^oI1Ulra-&NFO0jm?Zl z`+|&HW&IX$55!0kkAsbGccMeVr%rr&F7^G11MBO~jRWKt+IKPcv)w`NfdrqKXT9k2 zy_2niUe1ka-PHZ9ZK>5~w{ad!d3tv5h>8EzZ)A$jC|ca<-Nv2fZRE!3U79qo?$U(t zpH&w9Kyky{_}#|1`s#76(Mq(r&$bkJ&gYCd`3Xy$yFvCdhrD>;iVC|yyk-8=)|dI8 z<;4e6?m27_?U{7*Tr~E>i*@e4QH{$Vjm1{zDd~{GM_+>c?KWp-6TO_+QTRGUquu zH9mfxO9zsVe$o)DL2deLdb0Jn}A-;jb>MIK|wJsRYdt7W8@zJ1tZ8~{iUl#YfAnSxP z$)!M}6%HJ-!>79VSG%3?ZamI}8{tiIJ&icOcpPN|+X7@viET~yaAJcqAN8&QH&YnH zX821vFtiUb7(Um;;8W#xCL~0@{7=B^GW^zpMeW7nY{~xuUf(Kr_*!^FTj#9{yW+NxaD%5>Hh?x7hdMBgq)?bnNr*4^b`=OSuh9{*VrM zS~*Kgwf|Cas_Tm*hqCU|g}-#}#Nq8zjbHkE;6e7omoFl}+x8>J(S}`lJadx5W_TPBQYRKM#xYrszC} z_6E~}@(C2ZS)F06-v}(<1E*t40wF(b3a6XMcZBbP9AeemHZI`qpqxH=VCKE#^mz2t zFQKRI@Qq(A-v2pc8}IT{cDgmMi8_)6(^9P&SMmI_zePTQllG}rv*LlMxf6bZGd*|7zQ_I+J;{r+hP|y8 zd=KTe=GvG@Ej-u67%KhNbipJ(^JE@(tIswd2!hvW2>q!KYhVpGvfPGo`ZDy^Kc{nehJ*Me$1;IX00Do zer1!3h8(!NJ$@=bIkect^BVHLHK5mdbR(aUu^YbtO})H22d+3DemIYuC%PxgS%>!0 z*1jjLf-L1LtxgJ+z{3lxr^;{i>fJ3ThIT@q3Sv<>Ow;gr`Z7FTbNZlrD2B&dPB(if zJJ1vp_>9|37wt9E(ybHfV_#pRhw`!=bIt zhnsO cD>wTwe+aRKdB?{{?GrUk@Y(ynC3rY`1TT>EBSt2t>Pty=g``)bq7Q+KW5 z9iH$W8Y!djXOm9v$1r$FVVCO!vlRV`gsbof+PkFt&TLXb&Dm z`*9k_IOj}YTkV!v?X|y|Wpw1)-_Al0KD#O5i`|@&Gk$wN_mlX?bw9g*s@1cfSgsm( z5B-Jd@WpqWoQx_H9YY!N81A2J^;GZlg$jse>pca%e3@O)_MM@5dNxhZ$l2JtOV38+ zwM`|?i@jZ)O|tpaDesIWn}W59yHH9qx}UA7{oSm(^9DYzOYeTRy0&#z-9^D3AN3OW z#MY}z>wb1w?F+N&E+1I0^|+y1ufx3hMxNLaf&KiCslHlh3O z;y_UqhxcBpne_89S_GK?{ zrpnpVXj}FoziVej=X2(a9Gv3&rB5V;Z~l+O@Q9+F?og8~fbZSbW3^*4(MU&r?pOMV{J?xD@lOcvqFX~wg?b_>u z;iJ(g*FQLoDv1p|7L6*03{In_q<9)_c4(CO$u#=gLOb+BXr%Rfm*`XijcQwFwPjYH zz3+VdTbJ@FarioVX=1osvY|R3dFJ@j{8>((scP&T)`c6SOMZNT?0?s;WB$C&8q#Oy z@qZWpC0DY!?=qXa{G|K*9=$;{796cPc0m&`u_qN;bUyk-!sR9hY!Ka{&G(yGZSYR? zeB@~+_9^q0HMFORx!Xvc3)A2icu#v}niEy50TJa_v?52-*+^Tf8!UY|+DaT#h`t47oH9z2j2$ z(2|Udl?~fQ-_5K=3u&hq8Rk8^i93s>Z=TC@Ju7v;^-Sx&DfgrmdXREDgK7RlyYKRU zsQX_sx+%^+ZY!}v7bBym(ayAEl>^u3y5)3l<0{&nmuyWF--~wYFW3DBCi-#r>D;lm zEwn$jbEMcC8Ai(t5ecgp>jvTchUIsyC%FY5E_JWnUD`F{5F28FA{3jLelV&BMv%cED@WGi%;@foB=PGRl( zD}Czh|GtaKJ$?tez!t|>skH>YM^8MG@5|v2@qH0^2*US_$M7_3W6yZAZ-d_lSp8Xc zfqVcpJMsIS=fES|s~mq?Tf!zI53#G*?{NHgz+res`;YrBCNKXT_;nb&a3cMq?%ToV z(49@bo60kT&7}crMrDp?vpM;Dl)LB9$tD(eP}qYm?5xT2`rt;Vdq+Nh@YM%5c&GsG z?+9*WwBQyFD;bye^rUOGjbJXK-$W-PFVL-3Ua~YFnXi3a)oH|TkuKOb5BcrG#(dx; zs{tLkr;vFlIifPNnTs9pUJAUHXD{k5#7^+$qGjLp%nQ&>b1}Y*;1bWN%xBo2^7a{p z3)!`$(9vIWnydeEFD`3)Z!h^RS%V!t9zBlnzDu4#Gsl4CWq3?uyxiqU>1==DnRM|r z@a`qz8+6F-`8IR@NIg>Y^6bqTTWg@kf`~p)s#IHbGsP-7svBcFU2OrQ}dk+qvu)tKG)Kz0=f( z*M1gd3MtdZc_Gm}>az;|hJNnnH#m^*N_NsZ*G^iuuzHH@q;+?$H0@yTkM}#CtBYo+kK-`2g9xSMKnGQ-2QwJn5xMqxQ9IWe#R`gsy^zSHUMAc0oj%7 z7FsU8SbKyo8@n>y#aBD^8_At6yjSK}1#iXRZUgrKtA9VZdlr7~UOYA;r1^jNY z7C-d+Blj;}q-}g9$K1d84)tPn_Siky_`cTAm&1pS-}_eT{**pG?Vju>wf7a;`~D}i zXZQr)O*ww8CHQHV`s}`a^2?x+WRAB7dD2^~8Ti1K!t*%)*6ZC2aB0f8`{dppQ_foh z_n2b$EWP%|(VqNQmrQp0c>)@0Zfmdm44&!TZ1>%xyeord@}b7Z%?QUg!;90A1xH&~ zSU((n3c28iHv)W+iv=C$TLn$Nk~b?8{WHFXOe&Q;vmM)7_Pry|xI@MBm4GYBt7dGu zT-`68J=NGp?Z|n>!QQ~0TRXB|ILW1tI&?4p$fANkdQpMyf72Qx9;@S;Cg_LyG zu^2d-{Qi7@^2q#FatY-!t}roGDM`q3&SQ#S{5H>^fyqbCn3^`bet=OO-OGpg^TRpC z;k;qoqar>99_95^{GxLghhRb<%MOO+|MSY|z2R$MZW@BG(QVUzT1R(oqCbCM332-5 zYP7-K{`2ugxjL%1-sxL>lmK=mkBo0JV}^d$ zpJFvg58D78^-OgP9~+sfJFt^jdj{_R_^aVZU^Bix*4B3D5xXbTJs*EqiR%xeP08OD z&PCnviq&una^^$iZ`75UHhwkqE8`DcDp?1wPvJlNBgXfrHR|SK`8mjcLmll6%oyO8 z9Nk%gKG@jjmmD4WH(*C+G3yfNX9vJ4zoeh>DyHxUrHmQ5C_P>0+EiA4)du9D2VV`j zMjYAHzvtwQ@1j@GU;F)i>#Xh+M=obgbJnrOX?=dSA3O6z^NO4fNA>q{*5C2>?PA^Z z{BlnL=ZS;hE_k4K-qlw+ez|pwLw>VLALlCA3+k9m9N1*zms{tzrytEPN1iG2l`XSE zzoS2`vzO9dHNJ0$XRU%ww5#)BvHr9r?1RZamdlu8pI^sw=`%UrbJ|^M_RKOO*CMkf z@?86gpN6NZ;VI1}o4K)*{8vZgFU{-y_}|N+e>J|boi88BW1HZQWARu!^H#D}F*Nd< z%U)fujeTD6P=GxrmoEaGgOM(|0AAUL%^*F|@D4ocl^MW0u0Ajr>>FQp^s7^$?S?^M zuI3!w2jt^PM+ew8C~Xtvb6w|OGsC5=vp-_)KyhUBy6mnyqD*({I>0|!-wCFd;Sv7#fPzFf5$V; z`K|Ea?Ua{Z9qX6%WSaKLn(%vR|DX*X6n%@Jucya(eodnUy=j=!$VrDq>I&DxcBylo zt%kYcXPwfT6u>VSw!5F5t$jmsB5ALvjq}L1+23>Ww%Ew+RCv5xF$ers%!z0l8xwQ1 zzBXx{r!Qx}(20||>f_!qYV!bN&muQ;7H7*`x%oo73ch!b zbz9FzR>8qu{DE7LPbK(5;A8oS#~8R5|Uyt2anDdVP|0+>R9ISr+raYkz28 z`!?;{1ztJXh>~^UDYd0GJiPbP-*#xGcf$MkJ^}ALKXT@97d$FDM-!YorW7M5A6&dF zJb3jveY%4BQ2t2u^G*6O=Z^RFZnzx$?*Rszv4}pNpGr6=2i7+F%svT!z<|EeBCkUG zrNHq`KK>wQ`e{eL>?W)G*>>6q!namizOmoqc~a%Io(Av}91+hw89N<)+>70kJ%`U0 zdD5T91iSpTz105)vJ1XnQ5o+)qm7qL8w1a~>m0hqc(i^u8asd(MaEXjxL#pCcyl&@ zJQ8kn20Z0Z|C#$X=!@SYzc6WOm&NbG!`7ZK-&1V+HF5^*iGFVNc>isgmv(_;y?X&( z%C*1wOp5zHfwGKYuFh6fktcu3C_9wPf8CuR-P+5mYv$q~Mu&(=exSIqFtjPLdTuNF zgn53>NauO=S>_JM&EE5|ZuwwgsGa%xE1pM_!MPRf&J>*D#lPQ9T)DkRXY7P)Z@i}6 zwsg1M&xhLS??=?H12*f_CSy!X4Zy@Gbs*9%*Zw3k1=?33-~Us5mDUS|F^SK@y(b`tB4;ikeZ{gkYo!m3uOAPr8;HO;q znfOn!(*%Fz{pS;7JFxC1w{Glsel~d^oPts9q64o~Tx=~g(lfQ!6xUuhv==ODZw>HB zPxyvgzI=h*7azNcv3*_7V`2l%_`tt`n{zgOa1Ss3RB(T55V$L;>yLvGT=&6vtC{cV z5&17J03PWr@xH(j(8IE7%zk1LzZ;X`4`ir63)_B~laKRK*7KVfpT^rjeCWOr$}gO_ z#{wVnVXx+1r6OW3+mK7rgKwPV#9T(VbFL;SanEFAK!%C!bk-`xi6+oz=N~yo39f_a z>F8TMd*KV_|GHw!4w=1Q%G`pUq5XHg`y1sg=DAJ%xA@({*dx}8M(TxG5&oA*@JRmx^6MK_F|N5`7Y95$; zpo~YhuUF2dp2oESxWwNWfplq0cBkd;83N z^eJmzn0fvJb7cziFyf;ux)kxfJ(AZ^=KBHSX+K21={(fy`|6sEBLeWC{SlbxM^ih#B?k2{4g~q*1cX)G8GQ6G3m;%|s z=9HSEH_PCqDR#=7`P{kN3jd_=?gF0I^ZY`)_>rad(8b()I&ZC=y0}F-QIjs5bKhO% zE8eqHQ@OW3@&$160r^%>`uKy^OyAJXHsnew_f@FwI|(BrOS$Jh3%Z{IZQf6|ntSQT zPdo1<42$%l2fqKU)#Kk!7%}Gq^oddMXApc=aG$SrWkt7HFRwhUo4eCSjf7v{PqI$i zMO#~IhDR=<->rw=Xtue<6TF?+g72gfHv-IU$dXRrs=urJXBSfTJ$^rZ_`TL0 z_R!9E$or}@db^W@%~Oc~DgzGg-%FiBzp34Sz)r=!x`yLTbf!}SqqYWGyxgb5< z>>-pVhT3l;J_me@4q4FQ@^jIdJldwAzjOYr6h8HDN*L%Lp^h83v@Y99H2#sL6ki?uG^Q!_Lo znY`LgSsav32K~zHp^>5BMS9#lhaEkRc{Oj5+vZT(R4%2}v{8dU%e$kl4+p6p4&Xz^ zvhxh*tfFv`0S?56Z<4nr>dME;gT2$03p$Vcmo@+Ww^eoX4jqCu75(4AQDUE;!H@C{1>0>*3THicn>@OIfL0aTLnWNW= z{#yn=GcS9r_5Pyg{N%He&tVL5v|P4DoE_--i^c!4)!LaGY4C5Z=mHOb=Lh_2h8*4g zZySQ_!u|)&Y~_+6#zE`U!|YF}4dY|y`$5j|t>g?}BmVJ?>`|1n-XZ_?(Bh zJL%uv8vj?G31_pRX*bUYKV##>@(xBv%^i=9;MI(wf_Xd|{An%kA4^)~RCg@${eJ_R z+#;GRutGz?PxX-C+D6V^uK_2OEUCw5W( zTK$jhLv&ef#-0{QVeHL}J%Ary`6@Hpt%AJ6#oI0JPRJa`I(rx$Ea2oL84N#aJDs|9 zAMa=l*qNv2aZbAsKiL}a5!nAt_SKg@vm3i``Nzsh#~rWaq}zl{KOA?iNHmn+HVd6- zNvrYOu8Z9ps6d{A*SYdz$-n9YR>x199wCl-+bqtq86O|~Ex*CTF@AgWM&q~tD($Yp zE)vb-`l^?Zi*q3a&)X_W5!<_hi%dZl#*m3}{M+pKr!l=u_YOOKUKR%ly$I+zNy$I4ks+_x#XAr+ijQXbpaahk2fb4i}VMZwZ7J@ZbDq zFMdA1#fJ~F&ME(bM^E{iPl0xS<-PnvUQUE;aO7_z?_rh|N=Yp0-i;rzgFFe6?Z{yK zO0oG7`s*F7&f~;34XmTw0!hTB4?#X8vw!C&ZY+S_!rgPre4(_;U^wHFqVTNpVE8KV zD?RF}1goGlAy`nF?Cf_0m}_~^dkXaKWc~(>9>Sf!%;P%j7w`Tw@vHV=G>^+u@jd*` z_`=u4#uC{0QMpgHZyqZ@m~5KDKxt<-@08~O-ZAm|;Stsb$&e}Jk6g^0*SWz(W-l<% z*}~lkD%*iwAbAr6mbZXizq7|XvPbYb^->~Lrd~>?g^oy+>xoo0i^(gJT^JlI&s)8)w|W6e!hJ>@0Py%?rm4mu|F*ema= zCmVRlaK{!@-9d3epTfx-&pUU9NhkWGI%|mgR-G5A^QdG1@nxF(lIdB%dH9Yp=HG1Q zVJY+ReCFkO+{22^vc=gWwHe{+VXr%Dzh40PWp~u?{v%8P<~}j zk@g`BZ>h{+{IrI7CK%TMV;l4L7TOk12}ZRo-|hf8*_dkN^$Qss~UhZYqn#0-U zZE7c0u6Sdr9%6?F9gjOdu64)r#y*FCpJW`e0Z$;#;O_DhLlw-moeTSIz?a}%tzVB^ z!k$!DKOa8u3HAPydh%h_Q_p(8zuwJ9;LUIFmcp2;)yKa6vW-V6+bi9GwPlIQ$IOjZ z%G>nWjf}m5bv#xEOmd%-*V(-HbcQbyKPWg=M|jmfy?)2@rRJCPHSLSbZ=^EQ`0cba zE^F#*unuze>=tBQ zte&?C9u^#uUlqvwoI^WWBgocLV-NX59$yCVCCz7@cn7!_fe-y|A8O5$F6gJe^qi;R z@uw554e|rW>v7UI=d)I1p&QO;u4!HJ>P>d*Zso4aMaYr)+#y{=xd^)BB4WW7={dZs z=N0IQ5&Bt#EKxt|H`l9=j8RS@wJY64@YEmfU2ob!KibW2J)47E+W1W?tiDq4ovQ8K zN!ErkbUKYo^P(8NCI`KSoQYE^xhq`j#!_qt*_&<5C7mT|h|zi1pf78VH-Y0#M(52K zROgMYQ{n2#@wi&aSx)hSKWI1HHwewDIj7Znu@j%Lg7KD$R@o_`FuIbV7y45>YqxR? zp6~MauPEObqZjX_-m^R_KiEGRo^EV(ePGhsx!S~kW<=hA55G}LBl>DUWi#NQ3>dVN;UXmZtw->+cY z`u5`VPI6fcHXvKxbfjPh{xM!U)6rxk+~A0v}5Q7KD_oO(4OFNo{x#l9A`Bc zxS;*Bz^L&fXVERkL=MLFJCwSDZ6)JBkMW<&C)W3(F?tVAcz6Mw}vR_hOoD^!Dh;H{|bl3QI z3z9+?uwPJKn-m%e9+x~rY~(YOTS;5c=pRS_+!RBi=GQDrUVjH1nuh7|L zGiTvblP3h(t9jwScRu@O-!Ny|Jvb}DOA~Xr?>Y8?;8*!}_)|X4n)mK><}N@R>0=zqS4i#@juw z^ylG{TpJB1IM?qXgDE_;NNH_V&&TVEkhm)eVY_q+?^7l3gv9$OQ$CTKnB<1hGI{ME>sAe#Oa zKhMv>{YG#%r6Xr9HTs-FF<7r=9@Dcs_IJaO8Th(xrp59%Mw;kFgpW zJ-Eb6)mG?x$PB?}cqyjeIy3Ikku=7=89j0npTWj2e4(3cOZ%^2cn5pFEwnp^y~gs} zE4w$$QVfl=*3>cfT;y|XKA-q;EI!KEgx4LuF{W(-QfT{@?#F5SLv%;2V`^Kx72Ece z2L6o5F8FE%u*dtlB%f}BCh_y(IzyBG`EVZX-27Hi_~*3W`0Ze~_CuG__aXWfPixH? zI8XY0Mfqkt{r`_PUruuAm(2V(Lwp;K-A2(eJdleg*$Ta2}t#ZIr=z ze@-@UKED03c)efNc=+*T)sB}P`LT$;V&%}m(GCofpJ{P0n01NRl_>O#9lPX=Wc#TG ze)QCPi;h|boEZ7xzxU{|DUS19>+UneXV-B7arXTV=Myy#QK+^`3Q z9XOCDRrgr!KWcE89%*5W*K6!Cb5VG>0{X|#i3<$@h+Z!NMEsZ zz)23zV(0MuqCsG=&@1E{R8FbY*r#4z#VJRD!}xRH zri&*Iq%XGTs}?bVIqc0BS35C*PoZ~Y;a?H$WUuQxzW;32eaRE`|2q9AQSJlk8NTP) zW5jU^repC<=|qQSr-Q3@>K<+V{b%Iw`v$T|{@A)tu};^#G`J4(N%{M3GCYwH`53zY zzvSws{ulVCZT!?e{M8Bg zt^4C$I+h)0;yIN2@56+o$gVWbh2hW8-e7xbAiVPwt7q4xR!_zUcWw6{x@&tBzgG19 zyPnyIo>|BIl3u%idt!J$^Li(JreT9gm$mtRga75=P=1mCbI;6QYbZVt`Fk=VUn4is z0b6TH^)|&7X7C+dmJ;5MJ=gm2o!fio15*k9ar|`Zb18k&|GWp;TlgJqXPCZ_&wT%i z{}wQ)jROfo!`t!aw0?Zs_TI%-kDv0Jz5dJ9Kei(X# zgV@k*$s@wiIjqCP_vG2rx~ryjKP#DBJ=}j$+moEHpJTH>l@c<0|24junRg|GZxf!} zT#VQAJL^>W9IZ(A`4%~cfW^erT*kLePDh)#!fD-2r;*cSFEKnzM}~VBCKmM8TRqXU z@cADmwg8{!M?bY1KBA37z|_s}9<|5seJhFz4xSA$fI1Ck30WY&%r< zLGQ{spEWnSCA9z_-3?9T`)EC`ZhQwY7miB4wfFa_1sx;EAYNkkbewPZ9K6`>>AjTS zdVe?}TxchBzi@GATh$Bex&lAi-j$aT?drI~?)m6yyXVk$Uvtm(l<)L4ckl8wN8k4~ zw>{`?8k7^a>}w z_QMkPUs%K1p7e$Ab-bD#93J`}c}1dE1$)NcHm$o(G=z`$GKT$(Vc%lvE_CW1C=B*= zpYPUBNO0;m4i9Zm{j1@TQR($*myvHqZ7dcXW$?&oK6Cht;Zw>dozHAO8Eg06LfhBN zx8Uz=e}}l=g>IYHh|tTlxd3@>=3qdtXFed=A+NPCEEqo{e80oJ2XJ z>*c0tV(yE7&3uLLqp|!R?cFe*x&z8H_w(ReozJMtB<{C%^Q`D3N3MF$TPJltYjPwd zj9A-t26-XGx55#;Yw*cEfo;%x-kI2Yz|#X?aSmqB0&reO+uCDl#UAvJYI#fRL+)o< zR}s^&-9Jiaha2Z+76tcMr!8NldGO$IR>Qu9{?H%)^XuDlsXH5b<*gYJnqlX5vzD#j zTT;~X(fLI^ZBGHmGT^urzD*fq2uivPEI2zgkMBh|E@RdfHW?3(Y+ zisBEhrhhBRS{p4a>goO&_1*?1_-I$^?6sVWT-*Lj#`&`#&q~*}kH|2&nr0(IIzF;{ zipW#n{!#VW9rzn}CSSZZhjv7_Y;u|5uU{LzgL$^p?%DetyQgOv?LKMuv_A!({{*<6 zW}ZHIZn&e9KHs-{u7anmGn_g11@RjDM$#3y+cMDB=Gty>)p0RpuHb`As3P7dAK0ct(+xX^ai1r0Hm#^<^ldY{iGK}6 z)fV*afWEqitqpxP@U39??kkFV=HM$)+nuyk>C5lVC5Ka+WN1nC*?Xss3U@qc^-LA~ zZj5`j<_Gck#-{_`Br~HAGG9kR-(?m&QiQytp9JP1XY$(MIjwgq;J-gH2gT1{Vx6z{ zLD#9AQK=zE5p{BhnOy2UdoP_6UN7APUk7pHubnf~@pVY%NGEASws=0l-WPgTnl&KC zC*G$UmH0y6L*Lq1qFCf}S@+Lj{VzceC`KPBV!!%%6KAHiS2kkZ9C8JMH}fC)csKu* zQ`h`w?A81y{>b?c&F#|LJ;grT5^+w1FI3(dwSz^7at~Ujx2U?IZl1ai}lR>mmBN zQSJ3T8ynHS-8nq_Q{S_c(Ahjg_noVL3VHVj_nr8`A7}gY(Fy)I89E6kw=n)wsHc7} zcfVtKVN-rTFO(iPkQeS`eD_i3S+`H=49B}|XbjilZ&K{TPx|0Z4DB_0ZYiOkBVWa* zt&B5zO3?t@z8}s&=m55(aH}y2N00Tjl@hv+x_a)=EHPBd|C_kOGrD)6ub{6z==TA* zUcy=MG}@X++=p}p$#L&|UXg3VaV9PFcgl+2md5mXCU*dv{&ppVa={%o^Ni>nN9k`% zLWn@022*}vLg)bSJ&3UZN4kPlV$qwJT&)=#E>39uEr z@Cg>}ZJx+j=aEOFEkD@vGQX934W0vUH<|sDejNKkU&5Cey%_lv|8Ai#)R@6L>5wrv z@2iXar~~KHmv29{7>nT^VchxqB{8 zwd($v{#+aJvC70Vrh8}V^3adeMjiVR<+3x~HXf!8`F*m%pvM{Z{D(*f4*F_G&wYwwrmcvrX}5$#g$~&cdg$ z*LM@^q@OrJ`>WDH?ao?aZbnxGdw%}76L;G(DZPGws@1awyk7pUbI!FX{+w%x6*{y8 zzDag$k160i&z{m9PV9a*1KnXC@1XXd{u2^j>zJH$?V;<3u*N2{*7{j< z1FXHoY2J}aUN#?j*|3ca4DciP^1oJ8z}hf3w}o7WKgN&yS#qMywVHpZ-zW2X0cUK8 z0jqqmsGyGfWFHcrLZ3i5*qq1NlSL;N=arqCQg+QzGDvKDr0=J8D}ZuJs+Loyl}_lbFT#th8|&6y7|-%l=I;%`=Bi?AT1w!B@uh156WEuYUOAUMtoE42S?m#4 zeHOhm&FYD|Ik9g9KLOgGt@iJH;nBdIfk*R-vY+-(uUnw=||)+h1Qvoc_(k zA6>@WyqJ0mPUhas1;OTr25BRUHf-8BF~@FL<6HQ6U~I*t=-a+$njU$itM2gIU6y_I zQ7Dd%@`7VavZ@QdVyHr1H_$Vey1;VCvqt7@66dsZHpym=>1ntsh#-S zI_T%OFMP8y4gUx*jsvFgoN3bY(L8^Y?^{_1xfp$Y`Lc`Nl;7c+Us(;MtP8Pw&6`d! zv4eAS|3BW&JwB@H>ig%+1ULydktBdO5>QOQOTBQ3LYX91LA)Z?R@=vbeKbJThl*ND zl_XdV#8yU8Dz=hfTQj3+Q9;CBNTlr(wDu9SR%@#XwmuWFDsnNg;Jn}8KIbGSlS!a` z`@Vn7XJ*c2@3q%$ueJ8tYbVQInGtsPTSC7Vmu@XB+LrbDn(^90$zDKb;&Nyl+MUUK z;^5zr`@7ru&e}ENqw#|;B-hK?HRH7EHM4kk!oju8?`2sVZmEeVk8;B{bo2Zg&XvPn zTzaq7@iu!XL)3w}1RUu77VwresQ6iOwiXFzg*n={=FB2zYoSxQ!km*n4}O%*QU={K zBQNd9%ujrr4`1YsEITgKy6w2>2kk}gVM7<5w&EyqnHQZ89~AO#YCEOqp7}Reo7-5Q zkUznmzl}bPZ{c@2>;L}_`kv;~_k`Z*YmsYa_M@etFLUaS{lwwM(vH8UESEiH@)zLo z9p*l=Ik%L(b<9~O?oO`$Beqp4ee?6lXIpkr``({ZzuVId#Nal#@{&t#+3@(qxPGz z2l~MK)8N-K>@L~J zEztebC%W|6C;A=GXD)4X3~gU$ebT49wL7_y`cfm7GQMi@xqn7F`P$V+Z~J`V>e0W! z7vc&3{cC|oxHLE%9O>|{&w!r_hoJ|Uv)F68{rtaw8@o8G201pc=aKVjXu!J#nO!pG zps%n=7gjpdPWgRt(G16$T%T1Oy9QcW&`7%bf7xp%cq{NZeO%Ol3(d{Mp(g7+;&qeD zRQH>Ga@L)#7nMu~wcRS}0KnJW26$m{3NHvx1&sqH{m;SVh_Xk+(f^!CEf;uT zJbPG2&|ek5Pvm#YTVDR?)^O2YuI*&sufP9VIq1fwEDJ~TkdezLxpmd@u>CCRvt3V} z8RbyB|A$56GfYma`#+K${R#gS$27L0l@s00^UCdWpU;j)xzE_$`;9r#ZQPd}xz8UN z{SE(RSISPb`S`Zt5`1a-+kf=>*Y#TzuUTeAZ{mz8jaO%EJhtEU!(ZY#^Dg%na$hhg zzLLpD{Ri9r8g0w>ZRDNc_5U#bJNExf?0@cQzx{z5!`Nc!Ya{6lQtj=L91-*AnDJ}3 z)=(X~p&UA|h2FB^Yv+YkpYJ}^Y(XE~r}?!K9;z#`qW_b{J_`Ko`V#iSphK!y7pbSV zyfe(MvGT2GT?u)L?-JX`-V&cy%XVeX#V^{ZHp`iBZ$*z>Vf6lpc$ukPI-)Zd?EK%b zF3a^I8)&^sG1{`tG^*s+BH2 z*Sq^`bK<{hbobZRvyXE#wVCRv%~VKDr*^fKzj%c;N@Y z@fbp71+PDPVeUrYvWNxE?V_itO(lKi*JaQ-U#hvNT2TjXWdAR*2IY->!ZYpIw1QzceCRAIej~(O z79NXlK+TBhcE+Mo`nuY37B=zyHTY5S5Y+(v&kx-^`Z8>TSLM$i6S*%Fdv4Ng1?}Y- z)ZYk)9{N4kmDi=`Ik0Zk3;Z|ZZ#g3R6~-?aGjg6cmUeC0b>iE&p6A6gTksJpPP0}i zwkO_Gd`K|>#Yz4DCwadGm!IGG0Cd4dnk_ntuf@+}yys*?nR%2rlK=ejG|y}Pb)Uwy=OaXay@KCr!v=xcRxVa{E+*KH(+ez1BUPgyUs+Bk2^U7{g7dGj2L2Vke>TQ zaR{!J50UQ@GCnwSWygc?Gks$N=ElEj;3pr`%L_dZ_^Umf`MRL`_pFI|JnH*@TSA!& z>yQ=2h3w*f(R=^u>P!E>Y*f{Ne2wd}HJTmOXRWyJ4E8wO6q*=)fPEjbIqHD}`fS&} z_vNg86%2D~*bf^k=!flq-Jo~5|711K&m%512|TUUyTDjLn+|Q>&v>fQ1;ZU{1F@%$ z{n)h4edd|u#(&EC+^sw-yU+<^M=-wN9A0Q=QF8{LwjaI@`(56F_l1LOWTXOJT?tRm z&vf;2b&0E&cd}PUXMXs4xe~qnKI2#1C>T2l)`0Qxp=Z9&JbX0cYoWe|wency->fVt zJ2F~HEd3GgDV~Y!P!nNz^t=4moNC@5Gd!yOGUmNT@4ZI%y_G!Y$=J%0fQ)^_bzjE* zi0;(cSenCKUhU$Cty{bJL3}XId&Z+nRCLMruX%hU9y_M9t>N*%>`kur+ZLY7HAgev z6`Vt=T%iY%3;kDptpxXiF{rulnBr6=&`ZzweELJ+l3WxZ7lK)QkVuW2!JB8A_j2OL z@!U&{IbBX{p#{GYpObw)hIyc1E;J-QvtV3ljD1DjVl%R*%84zXX~73o7rg<_H$3>P zwd!)_$&J+Ht!I3yQ<+aZ;KAp^(Y4Q)7FAcBQ&j!}bB=d!F83;KiFDk{d&cg?Z(I6% zyU5=Mu>Xp!W}T>lb;gd6Pv@+mVETl{_Ywb1lj|z!r`gv+LzD=}MsCP=3$fqP&v)B(j=f60H+(gFZ#dE+9Wx(ZRqWM2cT)PVoadN1pe{`z3B-V8Q2bvJv~o4M@_S5~{}g}hv&7rqai>NgCm zzI~k9cazt*Y*6)oBmGO3zRY~$K>yTyGONsr4&DrJa_)6Da^>fa`C}*>V@1DoBlk@X zsGD=<_n~(ET&x?0ch4)-oFsc&-)cJ@e$;lc*LJ%5YX3_{rnBRP!2deqPnRQ@4G+)h z=kl-iPu`5Lt~k#p%y}($MGjcid8g?7uaQmmeewfXv)-N=Zv0arc?~1&kt+(6(=d3+ zea#`~Ma}}=4vn`zL(bVmc>h`KonRZUg)+9Qoo|xUpqgLRx!E=LkF6&o@MAfBIyu=d z>V2(=9!%NSw($A%d4=i5B>H6^6X{sYSZ;o+ICcy3rT97ld~G2Y_7V1o6}Mvl`|xwt z_KQpa+ZP{aiBGqGhcPT>47W0d#sD5RGKLY1;a0|QOD?fn^t$#l&jbE3JUEsU(t+Dv$t;M(DhAs_PNM${(VMORJK)AstgIDd=HGCj^q#y^ZiEgW zHrXFnfUl42#y=f#WaFP+9(+m75uvOV(VQb*JPjDO*u!?^;WK@NtuQn+^gY!=DY+;A zl#=?-opP}??zo{D*5rlwLQVF-nsjX!qc5e8c7>04aYD|(nnCYH?*DGSwLNZUtstgT z6R{%)1(!vegMlY+{?`xa_Yi(Jz!Sd*uHWnXVCgF_4Zg5u%du`<&B4S&WxL@=bjY3_ zOx>ae?9IWB)$#Yy6|+yCf6|l-^R1#xc#yS&?U{qnpZ%7a91%UQGi3H((GOd$>l~S# z*m5n@bRr*TWqxC>S;tw?pL`AW#5YpU>kQ(81>AEgM|9TQa^S6`&I_all>6QYQ*&!w zsjEB6`@24+bR_GRJ6t(+&!X9X1^ej3u0OE&v#Z;+wx>Cy+|1K%U6nenA^#iJL4Vn- zzie{nY&-1kC)&(^@r%~JQlEV)<=LO|e-CRsf?*2!$LOU})md}((ivRa3EX~-(i!Mn zCy%oaG%sO`mB9xy%pBA`|GAE_$yTt)>n<4Tu4y?|_h)<^rf2SCZN{&=+Wa{ePrs($ z-ek{$wvTQirwO>Ll^+YgX+D6@@0)Ntb$amGZEV2IrY-GRmDU^6<-;@ASA)yUQIX9# zUoMGp)=tMZ*4?ES*jtWnb=Mz#xP%k?1~<0U^Imp*l+oStWmN~b=N?%zM@-FgZO6Qk z(R}hqhB79_Q(W6|WHiI$3E|zqX5g4_*0Gy9=c(-YXW&O;Roq>0y@akcx*OPK%RBbX z-PgW(X2z?%bKd=VJGj15-ws1q$!TfSlV9MrQ9B#oKl_ur&yCfWhPKO>ymk=#Gs0Iqs~7=i zOU{Xhztp~>KkF^au4tbLZS&zbe3hb{jM~}L8+ZnrX8J~JQ3ckR1aw1bMy#2+8li|U;{yt#sQTH@im##g*T2ui448}h> zKQpW5!j7e{#O*;f(j!~!L64WB17#PHs}{fd3*_jbUg6|e$E6);#HM&MM zA(v*v9 z$sc(T815}?XkAg<@XlS~hMnUornMeTUR_nF;X%`f+uuF|OpH(sY1ODfE>+-o_`CeVLo&_Ex-;(PSyl=%=&hUP>K)>gR zgXenx<%8;3#SRv5T|TJ(D>q5st;Bx{tm3CwTiBs|lUn4ag?USS^*Q=F&3mr4G_?Ik zyf1vEwe!H&TSvILPw@0?_48GH8TFxePbB{Cfam5^pRd%8KKXTcV5=4Vg@KzKMcxP%6;ywr+r z**f~!=~ie_6*R8fI(mx6FZ@oo@)luF%_;bH|I3_yN1a~JJqH`2Alo{l{*}>FN_b8& zfa#IQqAK&;pv#=hqdT9|`fK4y{m+Q;ta<01(NpH}j_kmKZy$M?Gx(V9&z>{LJlpvm z&ztx5jGj`(dv(LD?RSGq+3dh?VuG?in8!Yn?eX;tJ3F~GgBbT<_-+XNHxxcR0)8BZ zu0E1lSmd-p6VauDI?TST>Riq#wExNjslw?A57t=f21QLJS+^$^2rrs~=N zt|4zbZd2?CKC6d*Q--tl&G%W*e1_Io z+DVLN|3zSatMlY^q{i@$6`cZKYToQQ|7FLIgqIF3nGJp}1y7fNuZzLkMZ{b$q*hUe z!LQbxiIWkVqekxBQJxJqz&oqL(V5R38e8jDJ38YuYgI9Pyk(jz!-{pJj^Rqia1^i! zr&GLf_<4w$_w`N-Z6SN`Y)<@{&l3mXn)Yq`cq`wZ-1sYW$erkl+xh67wDJ|Uqi-y< zR;e$=<>hzk*-we33MXl02fF``cb=o(Tp9i`u~Fa{9)FeZDac*<_e#?Gf_mmJ*l~jS zr^we=fh*Nl)!xYhXpK$Kt?#*>E%*~;)&a&I_dS}1XlFVny8%N7pl>C2|R z*Qw8W@7N)(ey9NMQ-M1*?$iRzew~H_VAuSm@haw7>G@h+HR!wcQ@{Ny`HNluY2xYx(Tls-hW1b7;befKoq+066R^p}b&=+!xHpI#57^zG*i z<`Y*Ip2iTz)i`UszH?LhmQHi1!yn{}xgKAPf|td|U&WVJ8)Lv{Dh~sbmrvNmL*s|` z$XcyQ!+s!qyksbeE`0K$)PdY1(^V3zd!9Gz3`~}D2x1;!L znDMB;>kdKBND58`uWt{C56VN``QS_sw_AWu&mIbn)&(vc*BZQL$DicAGRAc^pH%t5 z7c^_6wcsOF9z?st^zRB_Ebe_gMz2~?kx8?nNr_H-&xnB)7!AGHAi>nyPWvd zUc0ite?8S(TRf0tZw%v+JyGY`6OUQ0J+YJ;<3q70?zXnKV#A(!sX6kGXd-V`oi%cMHC3x;Vu% z#F;knPNEZ!?wWtAhsT`;kGb)&9?g9D9L@hW?36Zq7}ZVfj-TB4h?M7t^Ix#Y=lLPy zN|uI+w`F0UksG{Z5V3}El=#5Q)}XP`J=pEoUeU@>#q8IxQ#q%6vwZ(LeDDOerSH?! zQS)vPex~N{Jja?;hD=p^W4H0`?`Lce2p^qebLaIsZ*1$#*oMd7|Ap{P-4##wB=~7fm$-cg)0Z%pzWJKJkL9i5E1p23B)N zrim9UE*@W#Prr&6tioTe3xB1Zxx?5hV}SMN-WU!pxd5Eb!sb06-_*@T_S-K0EV&_z z_(shg+lX%r+_ejy@cSw!Ubc(%ubQmvnuWwS788r5zwJxyEEC@-rVsf;WjgnMD0$4x zWeMMQZXFekC6gZ&=8uimpExx(!>0c0X(OX;TP<_ugpmjPXz#3vezCS>|5$z4-kvza z>agYCR2)rg)sB7{dszN$K7G}4|J4cZzAW>8pbwo>VXoyL)!C+EABDW9Hr2jqxArL0 zZYk}`@9r5B<@}!%+RxX2e3QM%oMb8$&vU)!-{*eo(VTHgE^QmX6QeoH$jGpR9a1s- z3iBLiFmf-sOu0!J_#gf7L;6FT0no^S@2O)<{>R>M*qr-FT=3=KIdy7}oHvsjQ%AjE z*v@CRot|gA<)WWP#2bjczDe%R8|3c%ncST}d1t#F2%POUaA$J0_7*FL z0=+b+*~)bQ0DIOtRs`ek~Y~QARyU2geCZ6|*8Iv0a8qM)w zvY(b(ISZHZokts2bL|D>N%E|{hi~${;OJo7<$-hIT6tdhbmxPSvb+)2H$r!PKo;G!!7XmE79%Dwvo&+GokYg=zSh^ zKNtFc0soUT;+DBFe)XgILVfLn*QNVgK1y!<5_C`=fu9d2KJf9V^Wh#MekFhJANZoW z*S+82@h^yXx#w7+?c$f93 z{PFVCSkG81djCNA+pg{7*UvlD`EPgej&Lvvc$)=pnm%W~Lk-+OJV#?$$*Y(f=*$d)AP9{4Dea^pn2v`>FrT z&Hr@rtRlTv&U-<>Q95Y>@tXUY)0AuH*UGtiz2{4MwRn90YvRvkgLe1*41asio&0lm z?t2y<4%Y0M!Cno?&{JHW3vGfhi057(FB>}GbG9w%R@Yd#>l$()F?TId_ap=_A!A$OgxAh{bA5)%bwRp?l70G3xaY3lBLfM(4A3 z1PxWk+OPX_x#dh+yVD~71o~i8W+D^*zRgyj7vXDP9+XFNUY6|3F5f`5rD&-(oRl`y zpW$cPD8j}U9WDX(Bz9&hjuiV~ePBZtxyf38m~S63c>;Cg2Q^tyolRbIQ0G=;?#jn* zSWe6|(=Le>;B(7&QQtFC`fdvJ?eon@1ABKKIuzUkJ-EjJ_jce;#k=#lWc%o8m<@-P+<=yXNzI*D~w$g`r*T18=8}GUHsxddRsCWIEqqA2x+@o|TU% z`#cYP)}sF|q|GVdb7U#?`H9I2pXjwY%BfrmE^}BDEwF0I;p)nN-pBbD(izS@7q#mg z8M7Cdca%$3#`vFN{FBqPTgE!rH-H_l>{X+O11@jZ6i%lT!;MZZHV zy!>0_`Q0|TJ%L?aeh2xI;K_$sYkp-_7q;hlI7;W+Tt?q5o<1$FBA*^yN#+Ez=Gs-p z&&rP9hJGu6N7LbPzlTG`_JqSmc=EtcIb#4E2J3#?%D)m#ppWpl;11>l_Q5Vi_V!|D zmw~UZGoJ6xpWj}X&$@)2>&m2^S);ugHMcEjpHMlveF5Vvv}$L^LT&BUj5mRdibhTF zo_rGJ5C*T;(JuD=?6Mmo9e)8X>56o?(3odHrm^l|eLfR@%J7Lc(8omZ6U@V_%X0A}+Dqe2Pa_UNm=P@kROd|8>L3 zt}`(zLDhFQqzQqD{e9|>0l>@(6mt^dnE6P|>-dJm^_p_%Y@FYs1`tRqwP z^DJVNYOfYPo=(1l#X9dSbf>Oax;B^`2)<{r2S?Y+$ZwlIj9T1$&te~rt~F8zV>YK#Mf^tmQ0>ho`tS zdz@##&+l5l-ad3vez$UzgSp95HnZ01)j&Fh-xc1xR{&o8Gb=}rv^qw&SVdYlo)K=} zBwNtG-@@Ae9@hQGsU3V}n>q3re%FHgmI162Thvrx&Bt-DtIYg&4)(8d+-+lTT0F0M z8&6>OnBTM!3!vx&yY-FgXMZLko%C!Vqzo_K8g+Py_mBFmyn~g%r@&?4ts=V zut!LF@%hL|p6uIzZ{weD6@mWpJIeTO#m6!GAqH9<-`ar8n6*;o0_C=hWG@2GRWg6n zo=2>Pag1%HUKBVQ3opT28|P8Cfw7wW_ul%e@cOG&f6FZPgIk;Ze!ohet;@)Dr49AT zUh&O-zfC-^er30+?|0BYsylz<0c`JM@RxwijpewpDnCwDfV>6m&N5_RI6Tq#S(3Ab z*lXH!oa-M`o517^EG0*YbvC1KtPslssx53$OUT-6=o_}8rL^bMEc4sV4?Ev>9(LGwT*$uG>EwAk+PjQ= z7HID>_N_0IzE7cjKE3Vn*#n+N5VLTs5ihWJde=7gvq|4;T{0)JggS0ZUa1e%ahn0I z71#a7v+OB=H#OIar?2Bb_}d`+0-cy00)HiC;K=x=Ir05QSGu*^)b>a|I-9m*8L?QN z-8d#Mnk3ez982+*W98PghKn0)+iIYuS+t#duT@c>ftncNN1gGJeAFtMPOh2OzEwYG zF1ki?ARqPt;?>G$q#n}LmVw9^F)-zqLA&+&jle~}x10!_`dhK3ci1uI9j4oF;_o$S z36Z;=7^ghj^(7T~QJY+aVbI>th_?O;uc`0wLD|j2<4+=cW!%4-&;JYjADM#x0X^}* zK8XJZ`^GGd35WRa(ZBl#L@oCbf%M@qtUK3HaqGBSM#Q#>Hn>3eAtX#2X9ksbL4}R+v$!8n& z2urM)pW;j4*H-M%zU9*e7jo8&%q++LcJS@m;Bn~}=^EvVwS+mh<{I{>NWM;#4eiEs zE%;wN{{Uwk`MKDt)#T@6x7;0xsgF_g@lV10@$JC;?OtJ)jnfTgY_+~%KI>nA`F-^7 zKLs=OA2Saw=@sVJX|Eg1tlRbl^Wc90<}Lp$%+#CAJorr)X6i+z^2Xz-FjIrH6J~N~ zy7318TgLtKym6m_xhv<#>|aC2dVBXAn_xUB-Frj4_l{3_ zFDQpP69`|hGZu5;@$@pe;mm}+awf}qWoA=P$!MuUzQrZuWO|tqM zaxm~iHdt2UH09ho=pK)EVA#E59r`q zbj8Cg-m%K^qc5Ckuln@5pA+NkxgTyadSrwZ9;KYo0?VmU{ix~I$Xy}UL8NcQ&qd6c zMfjuS&am&+%C2SpkAuVg_*%`Z(Ky)1ntRPYH0~{DJ+{yuz9J_lv!>QNb8|J%7kK{c zTx>Uoezy?seTFr{XBhJ~+F{>G#|sPQx5u$(vY7X0V$16qXEN8|BXs!Bgn(fS`F?LN zC~WtyZ40^2EN8v7&_1g@p*`eYf3n@5Wj}X>Yn#ScMe?C;!e?E|{9egD?LU-lkvM^~ zcLQ*?a4*h#jJXrm*FH^d)Vf;gKDAG;`;FO)l+H(O!jAWISPpeI&{f0<_k+VEwpFfo z7Q`inU%SqND1Z+SbvDpw!)rbBp3|KTludhD?=tyT#P?5Q-81!Upkwj()13{}&&Wh> zd?osNI&%Jb>64VY%i|c+V#c(TG3_;LV%hPd&6v8*QrNSR{lV}+Lh(mYNqBiSO+I*eoG|yfG%tr!mI-1o5Xr|agZ)bOWNPOcN#v&Pp6*U@sJsNZv1%0 z{>o%~)eb)C#xJ=?M$?|%@!PSi`NAju`5FbtYOpq3YJB6{y^KwGO+CBgXSChd*&Xu@ z{5kQx(CtxRPmOiRMq2?*Qu%GEp-C6NT|qmkXLqcm@54E}V}=<|_gG{)e*C;gC*e}_ z!Bx;nd%XR#J8lC0Z}8FmXW-FV?+lR}0&t}AZlM`}c6w1De2cXZ>Y8@N*ekIa6(294?p_vi`*dRTW3W>lJ5=LLqo!Us z^rCHNz8Aqp-M1k*!o=DG;}AYZLBrHFlp%(OJ#Fc~(uS`m{J4E^-v09jSKhA!<{+Iu zG5%9_d@k3INWrD}sIPstC!-J5M_*@X=&UpMncmL=IlA+i-p|lDl4o=t#;e4cQqKa} z#vV%PGG7P!cvzbTC;8}j;WpR%_VbtyHI9$Tt3A{>+B%=co(BpCHrwCrXbB{->{apXCt_S$V&C38R|~^c=Y*6 z>o|S2A2Z|6j{hF`ZchWB^4hB4-*V;(;+(8Ux%)vC3-w`qsf_z(Pmp`BDR9q+=ben6 zuyyVgrS0Cv|BTmmZ}(sF))OBA{tHv!7Y!!Dd#UoMy}ZJ$Pdgt*D-G`jpKE{GmB(fD z736{89*uA3`blYE5DteMIJ%!h;`9Q?TTl0`7hdz=m>YmY`3Ad+?{^u=3cd>GlIk`M*rip`#Hlg12cUL`a-WY0Gc~> z$`{&aFs~^8b}_k;XJW74Obx}Gv2AX_M*qLrmdiAt{ z#DBv*ot5K#r^C&>!@x~t3U08mSCKAjSEc5na>aHy&FLfzr z;P`gz{QBhZ)l*&%3udpKolm(mygo%{4gK%~Y^&1r_d3u0EAYLL3SVFj zQXhTcR+VfC5O9VZQ`eIg5og%huxA9^?`04J%o^6*O{?3;7<3N)i9V(fn}q$b^b zeCa&&p!|6QGd5l-&)8hkyj6{E%EZRoGRE4yu4?(38N?8RHeMqg~(cOK4;?`AI7`JGMX{7%lrWqh;1qs~*kS@N_l96ir` zQ&Upk^tT9@b)Krl|Gv((eNR5=Pt|Yt`M+#lzvzR^A396*N@B)0KZOo?%Ieq$UF64I z{Fw9%>mOdM_(b42)Va2IGDe+E+ne2@b8VM+e2Hz=gT}={T|`}dS5~|0=YEEk-LLsJ z)1|(y28QO$?zUMm@g?C~FkA@??6t{0l{l4jy=qt)o5TAKj=Sy20?bF#J?9pkXCU4< z-uoUE;0?u6eObGJu_>=Z&-&kG;P5)?b_+lEie4q*=o0?Bb3l3=)OL=qcO0zle5Q9C ze9yzdA?I=r2gi7P@fb4lMfl-zK6Ch_t_L4#4*MbfraN!>O>9-sq%n;S$q(v~&iUTe z{aQ2Lw_bGV`tR$GU_Y9hTE>WKi3{F+yqFV$yxiqG2`=#%<=248Pb8%xh0 z7QVdx?=|?!i614snqucHrXJlYauv%%B{7HG8rcb}tdaaD2h;n1STdTjCi)FC&hB}P zhXu3O&ODFCe;0qGz$_d1PW3U%td+kV_#=Uqx08sAdt?CmvkZ+84ja1hjW zGk8XOztZY)Y=Liidb>7_Jan_~Q^tRKn(@ou@#m|cPC5}7I_-SMc1{O6=y@x8CEtp9 z_XgW#hdZZuunqQb`h9SE9(;5zpLFt5HYOu_lJI*MxVx(hcljy!?f4mR_P;DPR=RUt zzvx)ar6r;0v4%D?p-pBdZH`QN_dVV%KeD)+|Lx0IS(OvLU$$4S&a+9et=2iw>FBPT zpLfU+!Jeb94akmS_CAe2`B8U1_#gEZ>iuk-vXB!!30#(E_3=!J6CLZ}`;fZv66Pz` zSAAV5IsfU;x^$y2_tJZvb`^d1YFE*B27Sw}dgC-#Uk2@}bUbw*W1i9rztwP0`LyZY zyOVpW6)M<--@5kX>T&%4x!14g`vb12*5@gZo{|=u! z`J|NtA#9bC;iWR2qH?#`)(m_qqGMbK*nkNAwa+{W!67Ua!6cWBNq+oo7YggnoyGUk}E; zKXviD6#Sk7UVE?I8e$)oZT5TWEYMbRAFT>aTdTK{dPi~@Z43FT7P`H(K5xiE*69~Q zyM@#}TS(or7YEO(SxBukzvkM7v!bb1-*vRPG+Om(V8Q*Y499yEW~l;4HhC*5O5?tAI1;@A`Rpck+A zYPDs~pD-nhJ%Tf-skV(Z{ojovuP|rei#hgj?P_Pf<_Pkct;$EHu0;=~u4nr>q0d)o zzfEpDANa@ex%n7-6?+M(p~JZ%jDMgRzdQDY)O;H>f8mrrbKlhk!{eVfHZeXpwB1DQ zFo$_XxvTXS=Pnw3LCvX_Jd=Ow*D?D&`??Z=GoCZSwO>0^`d)R6Y;ptgtkJK@Ui9B< z)c#cJQ&NYh^E~EK`V86vimhJ!N0Y1hjSAue+P7M%c^Z5;VA?Ea@;3H`WWp!gIIC#FFlr!y zYt^dJJ__ol?RXy^ud%I;T>~SVmjbhF3+Y_lSA9^~#ri$LzT+wFzj{C1{dvK?i+qUp zd3NA|_nX&}_udvB6wfE``335Y7@RVGjR6>={&@7eb37w+UED^dM_aM0-#=Bd+IzmI zn|JP*oV7_-1h-u|nbTj3P=8qVsGn1-Skf~`nR@=4m!t2fC%nixm%X+pRm`3-ta?oz zxu#=ZacgL69lrjo$mV+N&u8B~uTXMO&bagWR{QYXst9rJbU8GRa&S{}XkOsVppjk< z%_wqc3TUhT$^nI&O03P~j2e4EF{zG`>{GmMK%w%dTb4aPRXI6r>?v&{M|m`Pc%nx+ za@4}v3T0KCn@CUZ(+JTH06wFz2R~`KJLoEop{(GcXza%vn!um zn(A?pSmH!0HUrqJfuS7RhPq^KO?&+pP6PD-Q*GBeU^lTCbolY;@J!*f5Bs))`8082 zB({Kl*c%aDKyD!YZJ$4enofc99Q_=@0@Yv)?71t07hgrjYsq~k|LY?se~g>^%ovo9 zm`svq3trI=ALW;gfd*FeEO?FYkF3W0=tzF&X}_?Czx%+S;3@x!6`K!Cg}^kQaqmLj zge$$j6kka4`YzWV5Z|(Yxoxa{{oad1z)XGct;)4h4itLLbC2!d$N!~sxX&e)6 zLf($>aRAR;&X}9nE0K!Zx!~4f93|^|?q|GJdR6%7V?X1WyqBskzxulDneILXeS7iJ zuDUJQ^gY($Q{Sr$yf?y|8!Dm6=NMaG;~A1>Jf(**9*g%xSMh%YJyXD3?Nmg%`)(fp zTX|OWYmA}CeGg{cZ#vL3UqL)3oR`sdw|R@bbE)%|!@MCKC_8C@@LJW2AIbgy!HzQV zq(16>pNk!-XZ(8K{v5RS3E)gKA1*z`l^uV6_TRhz3D*WY4qE7*Z~LXfm+$Ke;Cl@C zw120GHua9iw1`;lp zOXVf!6hY%j_3*3NJI6ikAqx6cOX{crC;2PM=&Utf>^--b_K|I~?{fYm)=aV0a>(PI z8|sH1%8r-3lw7U#g^eZF?z!yQ47RIzQhS14JKA1#745LspxYiR37|6Lv2!-dQV|Hsk4@Rtr>E5O&#H1OqV{}tn<-Pyv=dHk+r-m3)WsvGk)<+z3Nh8_f>m#-|~R`w(@MX zY(!)CxwU>}e~I2#0>5nb(ZEN%rmH<*EuRO@?&1UfSby)353m>boKb<)I7SU_RVXHb(7Jv*javLDj57mUI@+JjSKA{4*aUPrk0Y{&(Hy z4P(FOb)9o^FYu}U^QYH^$zjf*UPJdf(5lfOdvbVLMzoB4rJXOM4x1Uuygw1?WRYp79rTTy!fdxGj8;k;PJ z!d|Lf_{np&=s)+z4|ZzWI8*kMM{RSaZ0Gs0Jd=s;Pz{4zaOdw$`jB&h_4~iT|2}B6 z3fenfo^n3*WXj;pgG;*VO}Aft4s|KmZxIefFN6ma2VaE!upd4(>tg6z%{z{@-}P1K z*PnB2=pwt`pL6!6^j8djtG~(Ms<%Ep+rJ0j{hhft&)8tY<7zd9YSH6d6DqNiauC#7f#ym5e_R_5{OK%o{w|Zo@ki7Wmc4&9L zL(O7GdoAGs{I6DUHz|XA8SHyTzn%%~`+ybMeWPrUwc$3NRfB!k)*Zd;=H? zZME}3cwqc-`Wr|+fERgRHRJCg{!%R7CSID&^Dm&cMT-X+_bog_9sj84(6T4FQTqAg zQSARuLyy`$$vN5^AEZe=I;(6?a^er+!Tt2Dc4|2fQ2YDR>E8*+frHN17&8*fIp zryFB+K+Y?rK9 z*Z$fFeK-00Ytc>X6k{JCzsKF{I{XN`C_B6Bj1_-hr-T14zMrdoq=9-b)Q0HnGYdY5 zVB2Qp6&GF2vlaApHTz-p9qhAhtW|g?{kDyb6sm8im?WKNIMQvi@B;dj7ojD`o`$xAdsyUp-%+^8ALFYkN;mnTNF=>*Pg> zn$wI&^OZYSxbdgoYFyfjPVEWNl6;+_c5u-V#B4<>;G5%o4ec7uQh3G zqDgP?-^QAa=BSmxKOOi}WyG@V=(tCF=9(PMx$b(;&#~B3j~zoU#aD>aq~@A@O55gH zZ@nPFv&P5lCD-KK{r>&l^DL(EO!<(D+h?SVttmh|Uv^K2@BH7s93LuYWD79*Yg_)e zaz_5d+K|RMhH)xqV`Nv0NqyD?|kAL{Le}De*?0V6;`<(34`*8A)OAZ4!b5n5h zDt4=K6CPqb;_+W0L)Sx>R6XFZR#4vQ(?{ND#O5sFd=693AaI_V?fdj4ks`%HwI}>N z{CUpqd2zvKYnzwD?~8{7{CQ&3Gj-lL`O1gw&okGJr*FaA!=E3+IdvmApDzt;5&Zct zvgg^sUV5Fm=`wzI`tv20>(4I>ac*PJ8jSv4RARZDbAlh7riO@x{i*sJt;ld8b70#t zt4OsM8GlaGGZlu1Mf>jc6T9jrLZ8U#QQ1}U+ghnLfNomvw2q6mU89<#Q&kId zy;FRG7xxcQb2RMM9HnlCPqS&fpNEeLtf~XnaSuEr->e$De>r2W{w&r!{j~7zTIg)*18WvAQtdM*7~7rM??4bo^^i02nk33<_Prc(1| zePPq_(U!ZdhB1R94Lk4UoEXlC*%_{$8eX()>NfU>`24obY2WlbXNGU>&)V}}ts8AV z+fLzZ^8ioA;K`<{-%Q;`d?$Flt$+Kb#;V^=ZKIy%U+@3#sr?2wPRn?pU&CMT|LxSj z-T#}Z{h}ES>gSxn&P7?fOWQLR-AVu7nc8uU)zSW+IO49cy6wEv0{VoSxuTHJly)hp3&)N7G!C(unlh4(CIVbjLfu(QRN;)xvO zNqkYC$`{mBdKX!0p>O2}D_6_#ApbWh1_6J4$HXFr$BTjKBij6s&!Nr`(VD8C12Gak zV};LoqwF|q!|$huEtjyXmyOfqHo@qO4C8cuy3=??hU>KAFS?hZFt z12SXx<0JWFYaQi|Z8GpyfR{#emSUbeq5BnoPp*y(w9ZU`qjGqoe%YvY)$P$aDI<8k z_ECHL3!FtO*(_t6iY*p^=W_U6^MrIu6Z0Qq-r(0TKL`NvhLpU9rrv%4vnbM*?A3|BY6Igt?2J7t?dhMzx|cJPahC_zm#>$3H@Rp zoY+5>7)-6UaA>>cmc)6$!2dmuog14;jimRc<7<2EWzyD7oJ&|pd-It`-kWZ9?3qkl zhB;`0mzVNIJ9EW+&T1^=tVZ^Cyr?t3_MgrCbRIFS%KvE3v)`WDa$?1_YT`8Y;gRhv z;JSi!>{|N23w#*f{YP?xc&`f|UG=(c?)mk)E%5Ow^0Jqj+>5bO_JNOy*e?57J5Swn zHVT+o-0pEsq^{q_SPD36(jl+L37j*j^BIq&)-7?_?W62bD|GJUzJtk){&|v(=uT7T zEU?$?%^$O74b9;x=tw;#WKw6p`~e==3obcBBD$5_x?KvJ@__iA_C(;wn*QtYV))FRi55L8%BS;1q*G9>VY-3**DjB1$DK;yeD_x*7ll9x* zi&3X{!J*GDi+M+9L+TmL{jY)JPpIMMuR*Hzcz@{a=b>rM#>lUcN$OxhOV&^(33t-# z7345T$L~U)uEE9(F|U?TgD%GSlHm;25eA~`2eFPYn02Eetp5*X{XdLN7-nt`nfj*K zOC3AMT16v}{g%)eOR(nAAp20cZ zj}kW{Hap#Rb~VBKz`j1e;)LkesVCFMoSYcXe2y-9ZxXsC{sr_0wNiLCaYpqT>bSjF z&ROCZlO4I_iY4Y{7@JHrsS_(ad$A-wqLcUIQuMt4oWhHIm{xzqS+Qp>J6mT;QLoKye;qt$@_5i0 z3G!3@u_PFS*6tGXuvMw6*uq&{(l;85f$w=cx|23!gTKUln4k{5nLoC29x%Kaq`l3$ zf@uF5=jU0#rFRZ4xfs5@2>!efKD_{&WESf`8741eFE(s-fA_4Adl+*8b3rEOFgVa) z`DfGu!cWP^o}d0*&N+CW7&+J5uv?Zdm_KFuL~Bto*EFB-dw1)u7TbmIp_n@U z*@K+P@DTD@p!w0rms{5-AKw8SZy1B`pf#Dhf8@%TAOBm%f7P^XRP8!GMmdj~ zQR0z5jwQC8@M7EeTctd|gKL(z9+cqPFa0)Fo}%2B^(8GQnB133X+vk{Xdp zg?umIBb{~N#w_@tH@(Gqg7^?lIdns>##mbj%8}#Ak@mJrhMs=jiq?io-)KD1+He|t zHv(U@TdmvzZ=G~!q$r8KHU(K41r9%Dyjn+w_b;t~&e~l6aY<3ZK%dt#`;i3 z)Ul_u>w0;XHCg)o(W4@pYeVjua6K`BdfF@>ZB49yF0#4#U~;0K&9t2H)3vs6q&27B z>G|Fo@4fnC-S?aWUGFVtUF18!^u$ljiAl#jK1TIgIXmNwnl!vL(?<74{8s)uFP zXdTwcj}+;Q&BQgFGmlI=c2PTm*ICCm-&+&?`1{tXX5e>tNBl0kuQ~LoS$}KG?^;I_ zoy~KG&hB$sgSMc`tplc_6E(eJdO%9BTtU;e8#@L1)Sx?wF7E#{7@YF;D3><~rrR0Ov;F{Rg?A z&7lr6&deNdY)zcg432x^!+kE6-Gz@PA0O~q5O=~!1e`n&z=`yfj}wirn)Nf;4Z=-7 zaHI9{7shnqM$eSDCpU@*f6etW@?f3=N85Q$yyxrZeC`Wp9^cNcWe-8?Sm5DIVB|mU zUr&_U(b_6>HTOnSLsn;O8lA^EsPZ#vtF}#T8BsAU_C@A4=BWCM(Dyo1aXk87`JK{_ zrPNJS3}5pI^>-84d8(OLzcbu9XS#Ax*dD8L=f`UX!{_jQLUYp{W<6-550hl&isja- zAWv5Eyy$}r&W8T8XMw}|Z_vMF{g(8y?mz3xyx^_J*42Df8-8f2=BxVfQSC)un>t7F z9&n)8L>cXNxAR@Tmfcp#m|eRqH{SnI*|R<8sslH2wv9Jmi4J>^HwXUS3;(s=)|v|9DOHjt-L03eDtq;-x(Pf-3h(s zk_UMUHZZnzb~@ep-`GkI_j0e1d*IdZ*Y!u*t3Ku&-c;Qec|AGjWRK?>u@@8c?bMBn ziav>J`g!@O_-62fa11}K%x@VNm5myu{cqF0kNcx(C-Sj7-_^sX_jwO+=1CvAR-dvEY<<~3lfA+E1@lK%T}s{PBk7F@F{ zCWpq|3z8eZXPxHyocIWDF4TLe{4V^QLmRU7gJZx~HDl<6pB$G|_|N2>bU2v<+y&f= z1>nC5_;pS23vTHO*-Y>KcXG9SlSjN~wO2{c_~(22-=nzRCw#;bQ{j_um=3;^=qrzT zx-tOYO4h@Cyl&RK63`QGaPRxbybtpi&DvQ`ya|5(FWR_~k96wR~7VgjMg{F`6)~84N2faS?X*<|%ONQI71+Qr?ki0mSyBB7DASaT!pj&>K@_%>y zuJ4@YHSnp%-B)e3VcwdkpPTICa6!s=(#hm`;7Gi0VhN%HIYpxBRo+wBu#x8|l>QfMgpeMdNBehx2M+J9Fl@>;XwV>r#zqGjqhIJ zHn8tn&z~q6;T`QU^JV0iUS#Cs+MeSxc=YxnzgO$Z!p-#K%ZT48koy2)k>-hcg0qKpddE;#5gmQrrA+4=+mkXaJmwpHgLn0kJSU=wfy!As$+n*U@ZI8jnBhj0a z@dZ`eB_e&bY^kYBFcMo`ew^x-_SWu9WLN9sd*KDq<0B*Q(8bf4GZ|N3{VDdGuD@nv zK)!j|QNcX1?zsl18(Me2=I0-1td)#aanc~~`*4i~F69wi0=&ob%vXGz^WJhVH`dUA zXU^jp#XT+1@xRRD>MGyQ-2()ZsXa8-?QUoz&8~5(($A4 zHaxF5cIz0?-K*|q8I7}-#2TR3i~O}~B7 zLThlMh1wU6gYEyw;5a9KDr0ynzdMcv`)`Tyr0U?GbKZ4OxB57Y8rXWbQBif0+gbOL=$(j^g&U)X($hf`@v6epW z@!M$wQu3NOS3oqnjP}aEcg|t)S;ww4eCF|fa31}<*JfY*{q8OoZ;ilvV;Xqx@8{CV zr_Tm_QqfejdlI>pj(U=J%Xqg(&d|Rp*K?4;b^2V-e$XlCHzK>9-=D7&_1>rE ze@^^6z?-TE9!z@AruZd)f23=?zOGA{cCzC)(N5}^MPn zo7l$sk?Ku+%zC_PReVZ(WpSu`Es_sY-u-Xh?W-2ae?iyYYLOg+&hcxJyiH#l;GHel zNB*4iYD!;EsINoRBFTK8XQ&5KZgC^cwew@g+FNZ-p;C5i2(#ySDxF~iHHQDiX-ZeiTw=>_8`0gD%BN|`g(fBsen0TOSjrCPi#PaBuZeA)|=R_X| z->NC{W7<(ok@NYMEdCxjyd8MH&1VUp12?8WccE=-?)K@ex80XyopE2dAM1`8*id17 zX(!%tcmLSV>bPk!{JYHVu*pGd35TNDTZvgNOJ5hIB|Ip) z`Fvv7<06Ick)y5LvdP0%9Mzob#tjP9#gZ;My#ELi<3jy*MV) zp>?7c$OSq-ncSE-F*o+!sUu>^8*Sy@bR?GzEJ`ln`&K^h3@l2}Uj^@1{7vTyhp6$* znUHmEep>-~YwVBPU_C|-(M{}|zrVPnZGz1{i9ylAx_;5d{v{20{Xz|m{Udx^4Nalq zhJ4#@*mgQ`*Xu&8v7OVfbLbZuLPImj^>D}qIk#cw*^Se}qZ_A%h7ROf7I`4E8^Sf0 zHG~Ha=Kr9Ewh8CODyVZ&Lkxe{qoEG#)XHhrfb$yc6Dy}_oO8)Jvc^_UvmdSC`V8I~ z$!8{goX_Vdevjfai{BUUIfmay^SQ7gbW`KBT@#8s?3!a6tX&oS4s|3>4S|ngMak0P z#5Y47@2w7X?0YQK@#`Owi$o0>n|!tB_+8uI+8};zW!xdIuYyDd3Y)p5?9tQRt7Ye06x{>PxfYCiY#dD!k~S!;K+ zmWDScFAHz}3_%~(c{CRPH!m>enkIhy~HL$j_`TZrAwbBu8) z*X5tPwqMt}T&K;qtY0)={#@xmQ?qvO&mtWs(~h2>;C<`f546V5vtK`$T&?(=?x)H^ zmyI|mx__n{_tbNeo!5C!F&@?Pl-xee{{(p;tVi_RBk+zL{Tccoakgw#x3*Lr_OkJ_ zx7yKt_%y@?3=ERHcW6s%jy-Y(ujv;(4c-3-%i18n=_YXD!ZRp(5&9zmZPV$SmPhUA zpN-!%JpK$i*lI_cp+ws`YzU(fv+c%9aB;WJ|J2v|EP%IO?XWkx;5liZj zj-LtdXR_9>-=agZRBMZaS!*7`nlpJ2OOA*gBz9k5x$Ej>>`Qa#r+Ntcx7cGPy%7r) zwzrLF-;_*-V#z;Ru_SvR9r{U-!`X5oYb2`6kS0GjLi|$sx$-sYIFDX=xEg;Yb6^4O z%2p^($-{kK{nM}3wddl?5l`J*FqpV1>)_Pa*eE>j)!I8XBNhXr&ULFpzl)~KeXI0b zf%ja3dwY2HGH|B8G#42A#Iree`~Vv8yw>&=Bd6^hiC64c3+qFtQR6}1s-vv=OgUq% z%bKRP+#PCAJ&@LATlpPsNSp{Pv#gH9WiI~6HJhU~q3S^{tb3qwVrl=_{eys&HH7SJ z{u7s1oYKkmVILlU3fN!Gr5)aNymz(cu$TAe82Gc}GY$M*zUex_ujhmCU)(>YwFz7B z^L;$Wy156<(ACMG%sWv3fzie5bi*_9A>M#0x*|Ep>hKrZ;BkuSH(T3s$uAFz# zYI1;9i7WcY#?p^@1{(bw-F7Pf7xJ#Ii4Wwj`{%VA`^w|l6V#5!v*K0RQ>GnwHAXvH z_xLjHjP$P~7yn?kP#i(e`TQc@63_cEyak_VE%tm52FcUkgYWr$B2%63(dWy&Cz-kj z{d6|(rrPKJdO==eC~6~LS7I-e6Qd{yv*t*gSvH6M*E2tT{U;H#roRyTICV{b0kvi- zZkyj;-QQwuai3YQo=Gl^zs}Ukxv#;!&%5xQwZEaFYQ`HasNf~nK!;(p!i4TjN zW*>I-wd6;UGckWmNo+Z`e>F5NCpJ_~-O2KC=r-mE@*JaYRmx}SUO!v9o7lJ6lMTOi z)&i)8H%EZKboVN`C%&&36N;WPkaZ^Tv4{PV4(qc1cw_AYCjFM{=KZgo?#e>mN#^{? zMIPV3%R6~qzD{*XsOVQbuXPsDO?+VUac%bD@w*K#WXCh%g=cv`=<_*qsNvMlIy3W< z$ma2x%}34nv2%2@O&u!o*y`bhL7W|ZECZMh!L5YT3suS3r#%k&Y=D0 zigY~N&x41#=C!+0;NhO&Xh?yle*m5_;IN|V`KblSi!Iyw&en@6s;Z_|9|@ns-_F#L z7b#D98+_gjk89mc>$W;OGE9DEDfyXiGUthRoRRJrosm{YCj0NV@qT7D-`=|hUe00L z$NTlfY|s@Or60lJ4&lSB-v;op4WB274|I*O8+?51WqiVi-(K(Wnw0wki&LW4Q!h_C*vQlrZ+>YxDcZ#RaufFy!#6xtc1m;} z|K;=g|Nn#k(!u84&bjGiGdF#Wd*a)4_rA(K*>{k2iGK5Z^*KY zpXcBA`AfX0xx{@ZJAMOe)qBVR)cXE!Xy3n1KF;bKt~Ca(U8{T;@Z;QbPOob!1>jN7 z_`H1+u*e?&3_YQ?p(XYAQuf*~zZ_-W%Z=~lxku@f+9*BhzepFHdAuuo8_*?tkvZwv znU>Wt0~(UjzX4YlFNJBp+|y0v z$e_QMsgXGLmXNcs7~B8uMmL9>_I9Lu#$uk7O|TQ)Fs#Y73mjxyJ{h*Xp<6<8EV5mW z4w;W!7m&N0X>(S0KV+OZxoijSKZ$S9F^@AHT)S;zv)`X1R_!u#@X-S$q2-pc=UvUi6^XYo-GKRI)$-{>T+{f#~xi}|g1fZ0Dt4GqV@)p!c$_l67)I+1=+r!qN3aUY#W zuCb)6F-SX`wO_0Pd&STi8dqbNh#v*pe9o&D&gYMz7FK1Vvo`a1L;o)MUB>vzc}6)4 zLw?I|aI1NWJW%l3r6+y;*ev_G(?955_YZsh<^0Ca;lq)0%UI3W@=l8$#cvm{UH$&2 z`VI6Y9RD5u4qj`Mo-lsyAoO%kU-uK@Z|%h2_Q3P|n5U(GR9|-IP%GAgF1d1Za&;@$ zENVNp45cP3^KD`*a{zIxmb)_=62z$zSLuI*|DlFf;#aN9@C8dbt9z>z{Tudxbes74 z9pqOv#PDO;=h!q=du-a6ua`srYIH)-_iDfTLj11_I{h!#t{M@aVe*DX#_#Ho?vNVMz=t|LAHl@DRm)6`Kq+h}Q2J0ew!Kv1AMf3k+ETb7qudtJ^vkzO{z&<7bJN0#a zo4pHm`|x3RbGJKf@W=Xsoqh%TgDJ2V1N%q37qpKhi`mHdms;WPQ;P8s2EM3$$!*|w zK?r_fPtrna-&|m4zc`(I=ZeZ%pU&gn9Ih=uhZLh1Bg3+5#;^`~JMAwva@e)5HxW=$9>PlG<5z18F1c#r1dRm~C7AL7>t zV}1Y_MDvpk&2tRD-puoQMzD#epEb|q#E<8heXR4kJnimLJ=Uy?<;I_dx3A;9Yx(>? zrO}u}%SjiFdXtl0Xfy;GQ7HE~ZmB`uABK{Q&=8Hjt)CBpxpR2rFQa^jEiOe&4uF>61$@r$9+XY_NBPpTf0 zT~&=O>dUp+f9d0ZeTVoTPgcM$707lT`qn(pd@zGOc5AT(IBT`zX1+V?ymj(0wY*aF z?EAp4XVd6fvo{A_+ljL!z?#l3aP5xtbrx#rTX9_3sS~lC&v0Da>Nj!S^@{6it{UjA zV?IDkSN9UvxV0F5LcXG6R-*?ao50=@rk?ULY9_N5RvwCMR=jOF^>|D@1g~a->OdFF zFK?fVO(I|JZ2Wx9@rvmNZRa}Ac4l6j-9mrg!49TfY$C>zcRKrTi0$(JkBFzYF+WaM zT?g>4XXaz)wBh4qF{c*V3)bc3xzl_DyTd3x{ z8;o9?;<;wp%R~-RpXu}gc;HO08UKHd631#pP(glolSG;jI!$$h?~{f_JJ zBW4C23&4p(U9tVhL+ad`I>z7_p2I#WL*D&)@lno$6%KU`-X|WH9iPMa!oqbRpULv% z`}^nv(Y~G-UK#VOpI77o@>T14nlJQ*azh z)2}bj^GaNuvYxyjwWoC=$#XffTuyFwkWX!{OO}JQO#oNwb3XjR=Stu03BSfSz~hIP zfpaNvrt_B5pvB||7LbRgHeWShp{d~dcjg2>{ebwZC+_6?{3|(u zYt0yP;^P^^8`NV;mD^PLY4PT(e^!1*9;W=<^tbf#Ga0%{pIX5Avp(RQJCC!Uk?~Zx zpZXW!esAaD$^934ftxwotPNHPW@sc`@R+gH+4Bw0e4Fc%b>TCD-VvYos4MTyyB5Cb zZ-J-BeNQ*&{#f9UzZ|5a>O+UFK9e{ z<;~Y)3+Nete}%8-ZTPs>+XEi7Gh1_A8_~y=|K7e;uB=sf@_IP?GBSZr>#}1#+zaL_ zjSb#6%u7;We$DH9E`1+CTe5?LYdD$UcN=R3nnR#BITn%4h1l_`Su~!UrlrKqgtrAc zvxRF~1Iyy~Kx~sC1(DdGGmB&IongfW{>X}LBNnDMwh)`FfbKE8@I&10Fh4ygl+3wLY!;jcxjPG^s zVf$(w+t|Tlh~>P)oR%yN6F<*l9b+JCSA)p?2xGf<#lm*-PV025Xx~sf=D(|RQMeYZ zlxVqlqG{)ucct#h~ZV%wGRzHGZck6fGn zIO}2!``HG9TNe*!czEdV?Zq4rQ{4sa+1Gl~=R6$v_MrH!joK&bUwWn*U&N{fex6s2 z2F2r8dskjvbeQ2GJNh=?2bWv`PG+&T!(P_Xu(45n-mL^S+3d^cOV>K*G-z-wIT?Qb zq1rrPay0Dd3#RSt_;2B}n|W_BAI7uN$v-`6m&yNh;%|x`TeY5%LEo$+F6ocmY?~aJ z9C*=&e-x9P0UwrUSd&x>DuQlkC2r}khp1(s)e-4uZPc2ZL)<(UzI~9oe%0jg@!!PF ztBFRWiC(^c-RZ;p6a7ZKek!bUo6OY?To0)^jne=i8w7wODQDZ9)MPfGZ1h3~=I!8-&JBCNk0rx#(7Z&^i zUl}`Aej$9QdNT!FpUK>_06%;t^%V*Wm&WqYkE$&bu_K>KHwvdY^sDO*wG^~QBb!F+ zl0dYi@EFOD1?!jCGyhW_-jScDIah*G~LBxu%cW zGoznUn}>JI+DRGDTgahVV+ku)(mhYem2W=>s>d1!GC{r+vJ(cb!)mLS^%Ax5G4NQ7 zArZ6#;0Hek#n6Ja9ns#Uch^2+BZI`t){zh6$I|xs?Ga0L?TYSk(?hmL{M5AP#q?Wg zF9FS&Q;WpAP2j8&+0Xw$kzXgb zSC#NmEB~wD8(j)SnT zUGZGd7Zv{Giwe#^yzSaR)9LdA@D%J<^UG`0&`aIF^WxpIo3X#Vy^+5CRcFT873cTo zEdRNO%=m~Y(U-p#EnT+ZO&5maxc(9_r1N9d*6=?^VppJpGn!pZ^qNpFG{!PdmilyCds~F zS<0AdefYq4P6at}#wK$2e5?O@XdmqVQiGq|_@9wUiejx=k%k^t1Y66{r2Af`YkQ%| z)I-qZOphj8=+mc(Vw@LyG}+Ac@)S5+UEE8)GPsPuQ-`W)zNs}0pAvWNjZYtEElT?I z0pK~OFCJ-$#J&lS+)Q4a;Ss(^4j?WAk8FD0-mZ1=eDJp){3qd)T#rw3Q~2bAXmYjA zI_Tt+*4tb@A?G~RSAHj|u^b+syiU7ad@>@w?@!6q20nOX6R?VRmcb|cfKzJ$oqQ7L z|G#NNygeIS*x-T;n{IL|`bZy_{gb{~=$C!_W-q*@e*5-wiVdB5`etjuQ951!Eo$xL zVQ+Qn3UA-~_skfxrb!hpOenjo6zyY@LJ znG6#Q*xu*eh8+H0-7_F8K*f0d1ueN2OY7`$(;qKB@0P|DXul)uUR zP9xuu^`GUxr)dpS^ccs?YWys13wKPh$7IgT9yoZZx?i$m%6$Ge#be8k_e^QW@4fvZ zP3c2*E3#Te9&Z|=?G<=_1)R;CO?Y@@Vc^Jw;=q=sgrHZC-#t}#c;jB*z1vZ_^U;O6 zlVhC7b+>KL?%;2B?|!QC^+$7fpP(n~cGU0Mz3^i9TVCETqP&x`Gq`fL?cBYva?hiy zC@;^Z%d`5OyH{4e_UQSRQ*~+2QNQz12kkj3_w3eyQT80H_2{{9BY3faAHj_s+!PHQ zau#kj?Z|pr;Uz`eTg6)Jw*QylB~mx(;LGS*S$b;7iQX>s=|IMP0xyufN{^FkqzRcT z^l=CtvUMHnAogW6?whz5Kap$994=B<){|EG0A;1zZ8j}ny z@Hx+dGYp?6k_{dgiO-vQh|fk7&UfHN4@QAAjq$z-nU}&kM`X*7&^xxWzhA5LqLkqG zS4m7MKFG+su}a>h25)}Quzgy(*=S%9y(m&owR8gI_iyTZmr+)}V~zLKf5g6TeG@sB zg08bv*&b3SK2dqkt3^ws8eMA5HO+?>* zK;N>u>6eU)^ecUyru30_%sw&a=Inpt{(!(yhFq}LisBoC@6wC&)jA?I_}atxzDDVT zobx2}JsaOcSwk54vN+3y@-p@fz`CDrH9S-i_&vVxpE3r9Z%0b-YT*4Cnv9&6a=yUh z);QqHr^W!CMCBX+-rJS_los4ej<^?rTlDNylb-&Im_X@+MNdBm4$+??aUYGI9;Iw2 zJ*5SoeYbss`X+r4Jx+9vtHAlk@FvSYC51k0rw`k>l+U(q^D3Wh7db`Ybq?mA1OK2b z-4=?^ukvScPV$q`5%FrTtVRZ`(cRC#ijR;y6F+{j6_jX>J=2`-M@mlI$67otWGB|5 zfO;#4Ps%GRd)Z69#nju#I$fUKNImxJ-;(p*t@gG9^YF68cdaG1zdrj7Kl!$UtMv-% z#H+Rau(HJySM#ns`!)X}-VJl!x3467k3W8(oNpEvj2o!6&E?F(D%#1VPfnK+FBgAF zVIWtJ+nq4h9azly?cUe-?5;=dKCQ=l1KW1(UXASC^P4@pFX!DV%B@6p$6ulcc2&Od zXxt@QAhK^oZS8a7Cl+ESb_Yh5m9>7PC+=QNThi|9dLQqz{IB2khH5wdKlbc?ns#5) z`|f_D{`K8`s@{Cm!8{cl4Pm~;Pt<8g3ka^Zf7(7ra3XeS3s(o}%jKHGyLw^yT}$Zu z>g?D3z!)qCFBRZ&0q^7f#g2`4?mK1Z9yw)Y6U!;*055TZ6Z2iPb)n3GRF~U`HKd;o zGr{efftyMS$axH=2d_>nvE}!t51rz>g0X<{*$?1W=r+f#fe?$>~(2C&j{3Sf5;URY` zINbYJ_{o_t{JcaT#9udy@?G(huJCg&aPRyI{Dimvy4n66;OTMjBylN5`xz1KXLz@R zr+=jVW?Q<#zu+l>`(M+~?tIk-OAHqFYq8~s9p)8mPm>N!@Cl9VEaD7{HQKW5Uic)d z*ptp!kq=oT$~hO3mwy7ews-b2b0K3#oeIz1d6iMOkUg+JfTwp)&TW0?hwj#$9Xvi$ z*^0I?&cd^|)2~K+?q;u*brt^X=6PnKWHf1MsPlU%}i-gY}0&gzXV?qy+ zaOOt9IaJ-J1fPWmOZhvM97_-0q4rf%fF=bEeq zI+1zVReq)fkFPQ849kH%9A@dCBcc1;Ra3E-lpbU+m9@K^HCqiXO0QrJVG|eops{ZL zMz=936P3M%b+e0{apZ(miw@rt zybR=fLNh1K@}={2-xzb;EEwv|?@Z)7A5tz-UtruGVGK{yg6l{*FZjBReBQT1>%x=D z6m1we=r@`4Z{%`K4L+*AL*JvE=eL^vcbDVk!H9a5>OL))5N+(BEj1?aDQis15{r72 zb7$|Ly3&r|T2G{|UHjx659qll`kW_J-K+++p*Tp-(tm z&8^L`+W6{TjtPRZW-W8VRGv56wO2%^X5ZB3BL+ZXvvO>%@j_cU3%7%z4L+QwecjIF+mT(6EVuHDx{j(FL77{Xj^q7OnlUo4*v ztR?KVO(UPK-G`xp`P228$ZKpmZ))!+ioN~KqQ*(Sc`=?@vGi9x|3P2lx%&#=CZ4w^ z>g(%PT+~`OMBlrMGnsPdUE!;{jPsawK);twAzqrej7?YgrU{I{um@TyVvTc|E+aNw z;xb%U`Wg?B154V0XFE3aHuwCx?r%``cy?oNc$((hLs^M+Qt&lQ@l^tcf~_vaCu>{v z+;x?&kleoN`NUM;+td*rDEQJTGdsbUQ?5@V&miff=fYFg86>H}1FPFN{2Ta^@rxDqfTw6WO$O(!{lO{pAacevL2SxXH8;9V*FCHWeKN<(K2;=L-#$wZ>?G%P zjx%%oPN!=^9kk`t?|uF-@?+PRhj-4^Ie)<(7u3-YE~kI8hZZE~ZaH&f8ahA)@7?;e z)3wO4IAED>^E^`G$Q+*+>zJ_H;Tk_fk6(3^`Uz)_?tB#;@UXp)YD?hE1x8sbx}ZyI zudO>6#g4uUI9u$Q<5$MGCbZZb6Ewy?4_HH-Srhj~Xs76hEu4ib@f@;;DDNHoMr*SI zJ;}xUXgZjbcd%9wKQwFpf5I3Uw2*Al>137T`6k9+J)hIhcwTEfzutOYXO`EdsPfaS z=UR$UzI2MBr()~*Ym?>qaa0Rm6LdC(=R#k#(6r|6?Q?Q3u)haA$k_dvU%CI2dH*iI zLQCO$ib6wg@w}3{5^Ju5cbAs+j{0xzv&p&Tk>{05J93_%%rV&uk7KPfoi$BdTI}p2 zj9dAaM|N&h^pOy3Kvv`=CXLUEb)Rei|Kc0JFj1Qmuj_j&bp1r$(3Jxs>r_&1>cv_= zk?o6FKQ;H(#;JMTa;7II@ekBE;8@5fKo zb0!euvC3gfnjqg^$+=>S`SwP>9lC7efbzSuW+%`GS$`efq4~9csXn7USjeQ%YloD6hJVe$kvL2ku z_r8G~y%t{{V|S>sN`h7M%YYz9PJ$ z5t>Yfp5@s~Jo`Rvwik<^kd1!fYp|TW19rntC?39o-Q|&KcF!Y+;n{~{GRM!=D^3p6 ztNnNAajQbej61#*o*owx*|S@7j8|hArt5r@G5l0>b*sNLn|h1zg8=Iy%f~?Gu)r9; z-T_WuU&cN1nW?uts~t%)Xut7FMazFc&a1g%_#;63Moi#do{KJU2mQYR+7jApgoeAG zQ_xA@dKi8Gc~$#{*BCRAjgh**+q{!?S1Ii}!K?Jm@}t~Ef2_V0^8YVq3?EF(7bF^- zzW`3zE9v4#$e1)EOY`92A^b@=-yXZm^5^5vH18ChuLYi$%~-60e#5`-4D;j6hOb#M zAhIX(D_~LMHbi?xXhg%_+q6RSefUy1e`Ak;_?R5wL?seA0-@B{EXpdj&82|aDuJwEd&t)%fG0)2RPhJuJ^DFo2|E`)P+;8Fk z8lJ7@|7yJt|M``B^?z5*TJEv;?S7hP_582L4*0bE%Dwu(tL7Q*#Xk6Zp1r~UH?RZ# zUVi0X{ohsdCii`9eRm({*)jef)06n`tZ9*ZyZ-!0%>$15LVJCKal@&73< z?20_odDb_WFw~~(fZ;JYkvKZcSWKsF3m^Drt}Cy+#21ODJ&aX3YXs~0G+=RovnN?A z%lg`i-?7HLK(mLM&{@QPw7`WelX;i>$^H8_p?~DWxK9=`4@M!IbIug+o5nss9G9F) zTC}F3{*uQk>Wlt-$x?0q;Fq#R9xhA>>S!XvGG;a?c zJT382ufo?gpWtNM|A%m*xwFs0$Im11u_;B-$VkS1q>M=nmz^t?%c7ec=w@E;qzQS9 zsho#*7wGZDB6ETV$wTf z6Cc<=IWhm8afzu{8fDX#Kd#h)*W^6x^ro@|m!XPF%2m3~i3(`{NS- z>(8S`eE8>l`h00(Hf;;;@F|`nxrI2V%R7~86mvoFC%BsnZ0Iy|Ec{8nntbNNTztaC z&UG1cpkJZ;%BzsNHWMU0`?mkjCg*L~&mp|erU`-$N?94lQ zB`kdh{1@#X4}5}4!Ox|MNmuw1m(fNNWj|c^{9JASsKgIwH=B1w`!AvFP=1FbULBKg zi_n83sdiSZe%T<8Yv`aDey7g4WYE>XWzE0B)=eX_*KL}sWs-T0xv1SGzgG>imt_yq zxZYQy;!{VRg6S-h(uoOgx2>yCd#@~s!aP2Lgm zP&x2tNpL^nPKcSwS&na&6l%{k@?Bd@VZA+R`<4e**DhTkSLW3-i}=mvD$FdLNi4?9 z?7C4)hiR7%D*v+gco({h=mfc(|5_maxa2YkphL%9<9<0eS=&2Tx0M&7J4u;Va$~>1 z^#r=fVt$nll6|F5`%Y=A(oc%*oGtHU9ndx2L2UYCHFw*7a;}QKCc=#M<&H%9b}>=^M`b&@W>H@$8$nSM&~qIK!8-t^?)R z_e!7JHl=Yz<>cmH*aAm~#pTxiPTSjr3?4-L;;$yYAdR+ycRYN*v=_ET+Q7a)iG6_7 zX4;ki)vQ4sPR%!sHRi26w_qIDQy8-+Ul5F^h?%$Wm&@Ly_1!Od-Xq*>yRh&6f%avr z-agsBl{Jxt17a=;e7nHO2yoEl`J@703(s?S9zKTqdkW{K3me0F!B6-YCEI+vpbfE& zb{V^cJZJ36{!L@e*BPJEIv4wJu@R#p@jU0E@lU#;@#ZiZpK8+hR+GkagvNi?N#l2h z)3|$zZ*@=cbAdGeecBJFadII%=k6Jnx`$;N<-Hp3TP#q;uA(UFke6 z*4HBYCg-5%o#wnHKi4aAR%UnfEwNBy3lLtO4ln-$Jp40wc(XYk&FC=J|Ec(myp*85 zC}Wiajc1-HYO~(m#eD}(YOnIW4a^Zd>eL7!)Z3Qr?3>BC+(Q9yhkn> z3lHHm`=LDVs&_@7x0M(6-3HqK<=3Ry|L7@<=S#CDZZ^<<7n*g(`<(3W8~pY<=y3*g zIUV{eA$KV{#ENt45MPVOo{Gw2-_$hCcW+PeaK7|v>d5$i_JqeOSiaL!Shk%f-I%b9 zru_&WtG;_>&%XO+IXapCeq2^e>nZftg2~#6 z9Bd`)$+svu|HV%(F3mkL&aREk#oo3Fdxzx9Vc&%~3?qke@$a?utcy;FjfHjoI!AtA z&fwu(WZsJ(z0}j#tDBzIFtH+5&tF3uILHk9)3VT{;dq4a@ZT-EcZSG0?*;oFRO&izD{%V%ph&gY>hJuam z+3ZH$=Y9#^Ok2)O+FdwtVhZhiOnXwkf*AE}>MA!?hR7~e3n z%>lGWjG^yc%3I@;GVPv;73tctBG$53+Ecs&Zyot;EQzxqRB z+45TWMlRooyDw>Wo>Q9yec!G3WzW*YuS3`O;J{k>*8mJA&2Y}_y_C)*KM zmYy^wa4};-Tf`HXZRuUwnwVgWeHHMI$#>*M!uupXx#DLfRIpRLvPQS@5HAJZqx7jG>(W(gII(>FnF{EXOHx;v3NXwb1@G%z>+! z3&rH`CC8oV6T*J+I&GBlPkOefmh&&*6XH9gp07($z8}gzM6)ZuIQ2Xy)mKSuyn5ak z$8Kb=^Zg{ z^2RuV55K0RM)P}cuy#t@>lWV&aG(coeN9_0F#|6EU%r@sFIT@~)P87gyxF z?Yw)0=g)BoTw1m}VC!S_JG?JbUZ*d18~f!3f0VK=`svJd2U>>N1I|MBEN^6glYP~a zZT)?r9h&c{VN(J@U3;_Sxlz8ru&V-f!wSS-lX1Anw~+B^zDx_W;MY~RgS~QmyAIkl z-?hJ!K7FHu2E+TY0a*VTI3MB)f8NORO+0Jh5`4GL)uHLLeNEueuM}vuWmW7MG>y?z zj?bacQ-S7MHW)dSb1(BacED)PmYL*$SLdW^W11&ulWqc5Ig_Bk?Os-;$|cNpI4|lb z=cH`z<-NwVP*&BsQk!(U&?9wn?(H|*(Pv=SvR5alvK{p(uj-Mv=rQWu$(p(OQg@&g zKc50UGnAMA6aRQUZqET*T4?|1$NcyLc!3|fNDYpJ)~juq-e&x82}B#ypJ`A(+< ztFe_UoHvc~mD!9kjrhTZ(@pDK_O(sg2`~FQ=4G_=3L^112AF$-$JrNv#~)oF9v`J% zDm3~SIIISr4n1v;$i$tZZ{XLY>1ieoQ-bf&&pKO*w;CJ@-w5{wwa!QoAF(F(Eo3f= zu5k(1UG3U>X}Mqjs6z@Ut*rVb8wVT z^tPGchxz`B^FD2M0d}5jWIc@d#f{|2t{VLlznk)cn>c7uY~3FA|J=N8VWMr2PD zdSMHE%T4^f(&(hOEn#@<| z(UN1bFDA0=yOoyD)7ku8`I140p5C(NN)7%6JcRqcNMFS7NMyvKWx`K8{Z_gwch5g} zE!W}8@m<|De-DILkqH> z^dfw$6#Q6tNmF=9JIiNJV6*T-OpC(H2JmvN9pAf%TnxjZ7x5>v;!dr&nJ+!<89KO= zHA4aMl=FZ^bCDZ4hCDyOp%HT=WAh2+z@b3uR@}2Jw z?*^`yqr+9YJ=SL%4m?%5?=RR+%04h+a7v4M`-VYlWv9*igT1l&Qf~g#Nm(;=l~d4y zU+^%mUm?1s(Vu^`TX|A-_NR?Dn^`AGTjx6M5-;z1ypxtP37BV*IcPcN32e~4- zcqA{vtsc%wn>8HY3weU}nk z4qaBO`M`IlagH4RrJZxyXm_E)lks2LvT_+X&^2~~+sJy~Q*fmQn;EBQ^HqJn)*Z+o zAFF)JnkVV-nTJ%)k95Un1g@#z!3nSF3RhQNQ>ftU`b^*yd2I2TcNMQm4=$zsPtEf_ zq=#H~st!9x8opg!k~h^^Ncz`L^($40z8Xh0}}|Tf{j?#QwXPzooV`?*q(x z@h5OVXXoOtuiqbCcC=tS5FYE4^AsZ5Xi)H{2ZeurW_Ru><9lh@PG76ZR|GDh1vh$q zo#HE*CSOSl-U3|0SB_{==C{YtPr`K*O9v{o#rR&xV~%s?y)_4~v~&fYO*G~ErNHzy zZAbE7H4pZ?1E-*&JBrLaDT8&z|J^P?thp`*O$Aa8@5iO@N>JNfi7h40RLnLClN76PNh45gQL=5vwr zc09<_x8J%yAT(SCY$D@2_%1LRd^aulZQzL}AEL*jrv+zFUfLHu;caju`CL|fDRZ|o z7s{K^X+^}CDPBr{g-=EsGwJ`eYRoj>E5!GN>#Z#+9;1`b$@qv4`wZp6b=U(cE{L;h z`2JkJzb!_4F%n;vJW9jgS@I@`Y}?A%%Dj}Ztux2A)*M@*$;h#d2M2ZL*w%)PZ7k2# z*e0fSJEmvuJ2&5lt`y&IX3T_^I_bjb{}Jk2G#$zF15~Jm8>*!`a-XcKH7~Ew|EHJVn0#gT_X>T_1$_CchO{*tcP!l zHRvl1ez+BW_&D?_JXpr~9mZJknI6vHN9aqm`Ma9uGRAd@Y33NGDn0ul={qrVA{XT> z#c+H@>f6f|4$hW~;d+IPi>wKyO^2@S(Z9HQA2P}~=ZA4Fg)S{#W6{I+BXBG8qzqVu zF2c)y7g$Bd`9SMR7tR^Pbi3V?C_iB*c5Fv(p*rJv6XPNEt?zhrJ@pr7;IHkyCy1;m z8>~$dJ{=CfV<0iN)AVJRINYa&FI#nmW`$qnG)B+YS!(O;yE>P(+^?kHx2d&22VX~? zFXNr8uV#P?nHxItGMv6H0SAgdP^LJd%*AHE>!jb{M(Y3Yi>~dYnq@XpW+pbc@IG}n zS9(*YySbvtyC`$z^u0X)in(&)-mY_9?4_SvPabbGr?S8!b{N^uc;FKD>)5-fMjoV^ zI*;}4KJ#52wAsS`j7ENH=G`P2ABh8!HNJ)GdECpoD+4?qXG|^HwQQBbJIbIZ4co;O zWU7p(&NIfS!-lZ~xTDD`(M_;Nb(^E3r};Ua0(WZcqvYrK7rxzHevY3-)O$kRbJm>D zLlpbUPWmMFA&0Y*_pO$3V@@{9JMsZpa^0fu2O`GoZ25zZkAC_o^(F3B$qL6^pD8@Eo)b99 zk^Yv%tO~3$F9cQ{`671paC(wl^=ht~dx@VfH|W=*8LMtBGEn*=>rH`AW4&qh<)h`; z4|!h6dwDK>Y2lf`BlVk^_vgD#{bM8?QP!z%n%{^Xt4^*n(UXByWUZ33+f6;0zF2zl zPIRfMqA!QZtY7hM$!S)`vz_$GNv_T8WTACKo<7g}h0HV76fZ8~yz2H}XaTW(w-fVm zI7@5GrT?P)Iau>xBX4Wr{15g#+iDLH3u4!NFAZjIYQ;aamVuX)L?eE{diRX^u9~(3 zE56s7lburIWXP_1taH zh+)s_v)q+QBEp5L0(0o^?@OVpWbh+{)CBrSbQhAvr zR}B9E%OAj)ADZtQtR;lsy-NR<(!X&03Vww4E1?U{z&deBru%d$xRJSVDP#2LEO*#? zr!!9%xDT^48G7gd=sFzt(bqdE!6e!hnydjH*_*KP3lyOnG_vRGAzoYdldL_?cJ`X9 z*yk^*?d$s&bf^bhsXp0HJp_N%fb%=8BZ}5%iJd?xVkM4kxgU_5t^Nk#zjr4CD z^9Q-0=8yC>bWi&R(YLChl^k$y>6+2xMB_cu&#rPJ9H)YJM>F)n*hcOph`qe&QuM9) zy?t{yrxq*lH75d%@PL*s7TNnHmxI{Kud{OwBDhIy}m!ZFwasy1gl=E0D8M>9{ zlK0CRm*0MF>T-O4EZ-IWBDk`C1)ljlmwrz*_b>7pr$%(s16*$8p3E!Jn=GBFPW0Ms z?8|{i@ey#ao@qDpzii@Nww8tMOY_a6zS3c@)0Y)82GY({=2zs{3I2o!%GfMf2_?7l#eLa^xy#s$0IVt*qxh62zdPcqQF#Z7iAoar4MghgB{a2w|DC4_6=i%PO%NmQ}+9_VICg^X0e^^0}qk) zhS9$0O)tS8gtw_ZHFD(CB71l7opAaVnV9RuM$6jO;4wz7=>e*ZPF^f$oYe6x4H;mh{i!8&4|yb+s2s*~+l8 zesCf5IaFjQzD6<+!uvbVobyeLqor@Hq1|Zut=Jy+quUB>Vtcp=St)b3!}h?Ki0vU7 zFTcwile%7A?Fm2R`EB$om9~Ba?6=dN>x|V^ff2Q{k{D9or*df^`vSoOy@LQ7cmFF!y7dYnee7<^4UQwR279fYS*a6am z4)ltZ;92zf!;;HO{f`ed^PlqxO5|Mm7XDAuo&499E$p}6pT)11w9+39$35;3ir&8Lw5{5I z=Tdjzar;G-sWHmnuay*5W+!DF`heZldh0L8QEsr56W?6Q^$IIjXO?3wp1A$_^|U4a zJDun9wazUDH%0g!W6Qwzz9+n3v}Hv=0V>~MN?AF`eqgJ zDV=Su@>}(V9;NO`1q-=ifu$W7g?8rZeHP^@7*_gQ&{;jP?s`{@R_{{3dVQ(-E03f2e2BfV5})fd#?6+$(*F+mn=D?q75?Sfe08Am(&9k6`TmWt_reDQ zUsmjk!`40i#$Zj0eOxg2i;8{Y`0pglTY#xYhJ@eZX`m6I2xYs8F5{0R4n zo+<1nk^9lzR%?^@4@@o_A@L-gzN+F=Zsm;^y-n7DLc7fqHVzP;5yw0cdUoEB>vi0a z<4qvk{sVg7&`icEH-Cjc zE`OC@o;&ir{zd$sZq})w%yd;|mA?Vl-I`}gsr<%xrbu7h4iE7h{3dv&I1;XImAP9W zxm>$}Ga7s|_||YPp;?(j0@n?|Cv-nUeS4jP@45!fcY~(;|J0qHr1)a*VF>9|m5@Z!7Oz^wBYEn78DHJnwXP&kS%~ z0dIKgaS6-`;O{8y$=nxMT;R%a;|Oobjl;c{ zQ)Y=-ZZYkt_rP{UwU^*MLVM-O3FmGPoS(Bj`g~sP$=IC>K5!Y?UODk9xm+@y8hNV= z&9$WE_gsq&@XW2hf`)Z;HXAyd9eW^oOP6!bF|iWu#VLE+?OI!bgR@=+YI7!@DctAi zo#}O8!$>3NSc_fvU7z5vg1K@Y+kgcp>-C+jJml6CiIkha_DZDY~i}shw zfLqT078@?U;=VZD>2+7WH`z@r)%41_t@=1MPET#**C2+`#cc1)8WUbz^|Oe-OO*s_v!DkLUn%mJM*=@Tj{e^ zcj09<{>}8etoKFU&&ho%EGE}BQS{En#UP|R$K8=+G^QcquLT$k+v4n*2lm?dylr*bAl7T zH?HQQCF5!)0jHd0xs7Na63p}keKC-V03 z?*wb^Ss1L5vTpmd)7i{3<|g(HBRA})-zsMQ7BP7RnrfAJu|O$?tPH zeP6ZJV==Z|lIfN!PvF*r#hojDNc37;x6>eZ3CZZU!sylhM(PRA_=tKlnLE-CcV*?|dhV}F)7m_8 z-^Vj$HTN^JwYJ6bJKXrLyYhj_Uiju}n`6%sc;?dBcy9%=eg*pKWOziWi7PqV;BdUw zmIrRiZE@Zsz{c5A-cs}1<+LgMYFL!Eir^u;sdtPzC&&kX935Zy`9a<8m9cPZo+%`n z^t!6DT1)0}9u{<)eCaxW75cZ!r`{@{e$%JkUc3_s3m$`z%1o10p?49`G9T@3Cx!O?}AdzS5&I) z&C+5*Pg1Xl@&bDT@P*)cYeyv14A&FA*U!`ThM2oQuqAqLl<#o~9p8w4(g-a#LCY>1 zatpf22an4byS$t&HO@=6)RZ#@Lc5J-ofFWWl-1x{QZ^Yra~$2K&-jFzpX6QlUhHlY{!t-oHE>-B?4D%iXm71v&NxWW z`glF8sXUcclPfD%OkPWD^xC1Ei-6z8+R9s6>o;k8MTgx%-9D7%-iKX2Lwr+Y4JP@p z6`ZNMH{N5wDD?G_z-Vun(g2K4l>8LL=ic{MfvMPlDI6Z=tI~^(^P6ym7Kk6;3Cl4P zS1w}x1(p!790gZLz=7b4dZQ$gc(R){`Y~V? zzZzFBcfHn2t3Ot0%+VyRuQ$0$+pF8=wsKBJ$PUce*l4PJg{Ot~<^7+4U2u6Wy;S6p z_?SDOwegyx#=#tFGy5jFIa23of&S1!Gjmw=uhuD-} zlNwBc&MbXyD!M|>m+dc2LN_`M4VKvkcs0gCgSMRD#0d}n6EvZ3x<7DHSR1ymHbj@X z8M#k8=rIGlj(M6-`gSgvedg9*BgfB<>u&GuRx(-kA#!30_c;6c$)Q;L!vf72${n@R zFY_$7vf3YqtS#84ofub%42`v&*p8l15Tl=XpLr_$(@qTcU%}5Z2Xm^ooB7yvEqmtH z-zwcf&PaQay&1cf!VC2#<_F%h%=dB;MXCODo&5(lbB4@Fs1{aX? zL1;ks$A8^w?7QCtFX<`%0^6%*1tRd51OA-gFAMyoR<87C)t`;OVrlu!(iNNs{J;d?vK0Y^& zV}P|979{5AQ&r^Nq#6^<=FfG=hTAE<^8$UeW&?eS$6-5Px0 zuiQ%xLfQXxn|z>?F4H^a{~uKBp)sFr&~A5dytOQR{tL}B{}~Ts{&&{j$#;cDXP*!L zE?GaTC4vu}%|~O-qEWlcpwSPA3+fJTi$-7M9BjuI_wMTsf3*?#d#s1}`yJo5_~E(m zR}}?+DN*oumx(|7`QYzs(%-v1;GX(n@0uRs?-jn?6a0;ig1;Rn&!)c|6Mrwj zkI$vMwC{-D+qfY%@O~C~TCl4fy3AN(JGADvoX&0jy!-T;xybXJ*p>cf*02RJ?h}W; zLEe}v(Pz-Xs(a1ZYEL}h0K=&74Q_;0V4&UJjX9Z z&M%%N>%oF}?SzfIRT}ZCEwo>IX-ZAY5X1H@`y$xVUI}mxY!-T-6Pxz{;1nC+!7JTu z0$&Vm!~)v~V}LcufH{X}@v#GfH}GxoUzT&E_NTflzXRMvmY4H`jtHErAy<&MSM`B*1@8Hj7r0YsGv|xKeI%0ePDe*L zGRz&|Ocw93;VE*4jn^GpQ9r*A=cn{3sGouEEjH=v9<;TV#un5se(+-aY#2lGjmSAh zvIjRT))A~bU4fnE+j|_b)4hkqws&6~XX|k3&fRgz4S$lcD<=P%_^*il?-RyvB{t6% ze3`{gN?tH;^D{Al!v$K~sR>$}5!MCSZ3h zW?umRq84ald~a=UUAb+qW8obW+3PEpJZEjlh2~LrqrCaX%ouq?&lrA?%-Uzn6xZ(PO&Lp)Im=cEnNpK z2}}Z8Td~&mZPu$<^!Fg`y@T%E3T^~Gp@Et7OUh&tJ5>6ul$z>@Z#3Blp4I-iu=b_h zA+3xnzlRv#jrayUfp0*CMou^G@1e~K`@m4)ipTsp@%_D-f@|ns=IKF+FNOz8P9rI^ z3;0?tW3S^$ZErmD(!pAD#tLj1(9TqBTw)iLy^2KEV!H0G$NpD8B3WzuRqhKl)x%$? z$)(O*^3=s7&pF-1`b+G=kJ8`ooNRy58oQ&|C8t5}dCaE*_F|`vsq~9~k=PwCr`>GY z-DK11xAfE6=Ec5HbMwL%YPN31UQUjXt5dE!oz42pLz|Nr?@zErhlXKyWh^uAp42Mi z9Jg@NM9Bd&^}Y5v;C({1En>`St_cV%!UJTy4)#TM?ub994fA~X=#zg#KAQ0BZusan z&F3tP;G^?QKFV73Liy;I!dt#3ADzZ}O6XqZ<5_%^z2A8D(f;T7XyJMCQKS6V=A-ye zMCGIUfywi@3iXYXOSw9EsouKy!9f?yNA1E#`R0Z4(KhsmbMetY6h1oSe}Ip^L~fd% z^3fR=z(;=`*8X|%QTFR1`Dh(DwD{;slaI!&xutdb!?$2ci4T=v-;q7zBk;NFu-&+^ zEqWFDQxKWW!RyS3O;w_r={;Gfu@ z!uhAqQqW9ti(Y2wqMVcYipfKHCTn%2iz1uj3eiP5bN28r^gtZE6n?r_^iu4hd*%FX;h`T7 z(KtIaql+&3nW>A)^GIEEJNa1iL>HxfS6YTwWU1(FN*5jJeq~xr6kQY@LgrqFE{gtg zCW419g|fo?LGcI27psPa4u7oL;31KZq{Od4ff23hGNAET}I52jD^WDE8*Gch!NL!*yWv zo@hGo4$i7;<~u?&vhUyWOl&~Tdsq7JF!t)u4L@VeC3^p7@B_*HXXw8fLGKCpW8bL( zL-$P$euhpXdgZR9v-RF$o}EkYy$!v0qpA1GK0vcAb&tR_AFLD&!S8O2Yz9#)D z-51)u4%!u%CYn0$X4-lee)AJk=bZ`--a=m{qVG1QU*wId??$va(QH$A^pmEpyMSNO zbstr_Zc6A7^mLl`3|)7F|1RazFF7PLgN5L!&eCzQ8MUA@%9=;kF*CVDzcqXU`-DWt zb?6C;7ALRrd)QNO#lE`bh&?B7HEV-=(h|I~Mwkjt=GrsRb8k9b3jIu9q3w+m8h|cm z*x56ry}{_eGA>70w~FuOQS>SqyC2!Kt?RfS#=M;YA1{E8TYfBln(SZm?Ogm3;UTl8 zr&Rh&`F>-s$}QQcabDry?&IxqZq9J>?LMK#W9@Uo_2??`Nrb=2`peBeRCqm!HT;pu z^FL-S7-GE}sWVr=&m4bjpK}*Fb6KR$3>_G`#M9VMVZF>a@B16@{l<)5f%h*n><>>4 zmE0TM@O${N#qSRyBfg5~U-fb)&+o85(C4^pc)sY(!|>k`evki-;`i7$e#zQF@qA

3XsLzvU1@1udH!YH*dM@WB+n;NXBWNsI@%PTKSb;g#7oEH zCog(L7yEvOnYt?C{#lfxW~2pm^;$><`_P?^bV?Z-&_) z=ED=B@!O}qCco`sbAX3Nb}}h4fAM zZ3#RwUU+0THU-0;@amSE)B1SJ;IjsQoqf6x-X(JHXY}i_kHwB)jEUH^ru9Y-W?l#n z4YwnR4WY>4D7EN2LIb}$vOBRasG<`!TS!PD@Wyj@GZmsniBkya|I5` zc<`))|1k#2_9{FN-uLkkTfpLZ=fm%+pz9;TGicA148H@99Xf7CH~bEHA@FqQxZr&> zc5iuR*$HI*Cj8DW{0{v#s-1vuMdf!U43<7iQV!LY;&)~pi{FhB*rA)tfxVO8eTAK% zoATZAJNaf9zblI1cjU}f`X%Q`?<)b05d3G69*18Kv4OGMun#m$(ecx9$8Mb5i2i_X zc|vTz-P%{-$I*D5=zMkT`+Uq;DPD&h?c{aXR^fBxq3+~$l(Try?qbt{A*3Ue~W1y)vRr zpV_A9mFMJjv}fs+FAF|t$KrKC@MQ72T;X-_rMN!$*!G=!*1BJ<`5DujXJ77h+&HX* z#|d8&9=8$R^tQ$07>gswuxaRs(boF1&PT_aoPEPE@6EIGlzwROxQ8tsmjaJN_BJyv zVjs15TywWPPHdy_IQl1epC)_XE54>}^zWqO{dQ|NI-acS|Cj1`K4eC?ZM2JyCovjo zUH|{+ML_4L!P$B;u{kaEGuGNV%LaA zNejL^P#Zg>y?9^c0K3;g9;o>@IBE*)`U$a7R^tPii(NwWIk81dSO2?@#hsQ^=_jUb znWx`Mza7~tdx}Sow{IZE;KqSk>YfelSCg(p{8~LzmiO!L{T2RWQ;}y*?Vo-w-`I}c zaKP3-bZ8v8+mSa$yvjw`@G^p-MD0XqjMwk&7mr;kK2(Lz?2jUgkFfW>GPbZ);1m720QeS;rarK3 z1-2IKq+;U{|I)d7|4=FQq?`+VPovCX&Sx6dx3kaEwye88K$dTY9t5{S18a|H>pjp! zHv6nD+Gqx!j^zFl0|nm51rmE;vOBhNGB#0l7Ti{HvJ_CZI9HqVSA22hOcjY|f1h>Q z2k5mo5T`D5@#;g+BsQh*@;w<7DL;TQksR9Wqn~IQv(mq^pOdk0srK_}t(9tH~u3{Qh`s!K-<{WQB}Xe5icz8o$O^l-?bjTrzHzKc2Br?{|c~ zXDmA2GZylGIk}3A`Za!yv5@ySQ=c*N2@T0utn8QJmH#pp0>feK$x|5%fp58j&k-sc zOnqRx8Q7ZfvzSYH8H;xM>Y<*L(-@1=dnkj?$!Cm5F5_YKTiRd1c!*DP(Xr0)$YVUl z*$0H&7jq_GzX9IP@knDlQqLNX0?Mu!WZ+hCKL1eroG-}>VvKKp*3ZWH$eA~PWqfWu z)4qX8l=|u*VBzfbYW(CSzFgq3;wD#-bI40f?aEoX-s)Lr$9JwGu0rBFy|(D_o$fC2 zot_@Ycg_RXrTA3~{p;$#USADg65nK@4`thAze{*YJmV>T1LC*-6!A0h_?5)nnCErg znCo2)Y}j1Ai>Y%RS0dL)E{Waz0X{TpuH)-pLR`u){1BG#4SAOf+!EUvPg$|!O55?2 zo62(!@Cbah_=F2hWUK#r{Yvl}&MPJUIT!qTBxZqqGtZ*d$)$^4o-B6M@NzczPY&xf zi6fV}F8LRaF;+(>v1f$7xe>m*1|G2;KKd}_WY1zLF&>^p$0j$h&-Xg;{T{jaDsWdp zcW=bH+r&Tp2b9rl2PSKL`&c3V>DY0T$n*FrF+H-6B5@)gLUST(1)rONV>SPW^NmE# zKb1Bdz;c4$VMSWt4R{Lez^CFu!-lT(i=4@)YzuHF>B(Nt!V~IS@z7AR@_$bbi64Cu zc{du_J2}A~N;WZ3F71Gd6_NNR7cgZLBa)9C(cBV`qK6JqC!cbMfbAvt(`&pdC$^~% z-){jgLi^$4n8-LDgBFfM3k{5g@F>TPqrAB{j`S`CH_vd%m_E&T%6reEEt5YY7Nk5m z(fbjxAocvNrfta^R0up<=wmt8I28+$KrBdgmsk)tIG461R#n#Gnm+w>C1dl&@)G3p zH01PDUE5E!z5% zTrOhU=JUL$l;>)Hrho7>&n4c?;@Jn`x2}-*`K1M4Kg^n{*pAN7n{PXS!3ivJz!Z

Y_K)QD{u;a}uR zfyaohR7-uS7aVMqm$^NRIuaKsWtHwUJEKPG?7$aO^7o5BjKpzSbI{Ga6P-!gssk>i zFUi_sSVoPElSZ4)3+`&Z__A*Tl#dKC7z_%J=H z=w$nxbYKv?ETGOA)&K+1r7Ya+R(N6m8d!#bm&F};5k0Be?;UCM^K#%>3_PX4_R|SJ zHSrRs@FF@|gNYZxVIF0@;C_)+9vwz-^C@~vp2AIB=t*#M61b{+5A1@QHu%*z{Qo=| z=ogfge9?z{h?@fHEyy(Vnf25ao#qUBi|sIYJ4>$_7*u)<_!{A*TTh*1=+_^i*EwnPIP#~jO>1jRVO*;` zlk3650p{E9hptsIlyAl&Q;>6gPu{n09rASeSWTnf5y^%hjAQm(m{%*yB4ki|KFm;4yYrvna zeXo_6)4dw8Y#WjDq9f;1USh^17W6Q&pik!|)GXoINZ^w*q(jX2b=>De%fe%YALpZo zg~x;D1Ea)(mOjYZgjmoxVnRj6G@9kDSWqb^aiF3Pbr%b|2{@Y>(=A+&A^(M!UWa~R z$%R_zM)3Olng8)i^Ml$vv2BY@s|s)XTx3~b;=oY<^PXf zXZf&;{lP|?lAo~5%#HP3>=Tw;%lP7&eWl_T@nfz2-t_+7o8jqG;c*MOM2_vi_h}J) zc)BUq=F?UYdr`t?cMLORn_3Iwk1h5IZ+e_peA1UvzArQ^Ym_5D?#!K(TdD0`P5z`M z$l7}L)>&I_sbwwl7-yko!3PG@?*ySyzAtsRGf$FR;FioW&d2{S|4yD_5q34mHZ4pU4f4yHly-> zKlPWV5zk7$O6k+h`T(_t7rexQmvZnT^@M+kUyIZWfqTJ=tl2fn$up_HFnxg6C2g4N zcEL>%^l#C7A-It>y2cz>Pg%7_SGcj(=v{FmYxH9B%!vI%tE*$|MsbD=QeUkod_g#N_GkqGxI&~!L8x6Rnj(jUi zOAmR>K1m)3S({t^%BM|WMTQa+PTvNjS6F?kpl>_OzHRpvpI)G3U7T0yyh-1F$@~iO zzYM#{Yy4X4TFJMz4!Ta}k~Ql`%=-dcQb_U;yrK79^l);ue;svB*oWuU<1hA3dLLPb zJd%rVLkYNOlraQ$Yyyusc;;Y!^Z(J+{1;zB$ob`bk_)f_+7cch zdEFl7lJ9S(U$Un>5_sMOZ&E%yr(8X`>Hs4PRRi;d$4!bw+R+GFH~spYrZPWoZ?%Mb_4b zMs>$Erx`9fU<2fSB!_tJ5!g@+pa;v-;NswaAvKzEDr zxhVlg@k7`{ooBdi;PP<^Yz@F9^HK7})aQHrYk~Pzz9H{c(@qKZGb#Hlm++#Qlne5_ z6nGk;SAnaMd=YN-YpeIDU*LrHwy-9a95EH>W>2PRbAAskcdUt-ch=s;UHzi#9cu4_ zG0BDB37=7W7wZ4{?Oj}T*x0*puy-N)g76qiU#LN*skJX_8>jgl$$zEvgY0PfL58t! z!MsfXCrehiTSbn`SS%jB(!bi&2|VCN*1Vz<2rm+u@1{;PneRn!k#Z~GgOba-i~NW7 z&qw}mHP^HY(DS103ycVp|3#m)&zZ#cq_0xm+7}Qxul56gH;z7*(?^lXBDWU}{;9vj zl<&fG#O|*2Ecz(&yOcWJ$?uh_pV-^!r<{orF3V@qPm$%d_)YgzmLE#(l;zuxw$JHB zo88Is*~s#Tpg)f}zg9uxz*Y-v zE?_$bT_#m^<{2tN_AH~GUst}paUt@JF_HCjhkR52&riP1Ztj76W8LL6zhlX_71>`Q z-@wTdd*R}yXynz zA>Zal$TwNviF|9s#^%D_m~?eP;HzX?BRo~sYNMcG7i+r?*~U69nrzeH(Kgne;j)c9 z`yH~)7Dcv++!fhY3tVpC%GU}4J&|j*z@{OulA(E>wRDGEW8K`HTx$@y27J-vn%ExB zDc7K*e6FvQYn0h*miu2I*El1}+7mr%-L}qJx4qk>t*<}WA=lQ0uiLirJW{UxH*!$q zS~b64wI>?QhpMP=bRR0UAIce(YRz^jYc|f*vSeDve(1H>DMzy(O5Tx&*$e%F?1#pY z1HP|kmaLy;on`EYt~T~V84Jl&*SohyBOc5A8GB*bhy@FG=l(nsvms z{xoYV*)MgWpNzy;<5uz;lrkq8puacpi&%OoYv8IClS`|rCJRm0^K8i#lzE8!>eT5& zoh37+Of_YGJX!V>)}kl<{$kerd`I>vt^M`A*SOmn@_`W@Qu3GXMSn}WB3N^HlI)AF z_V3Z-)t*5j{kWC7ZdKpd7u`tTFCV?mpE$bGe*^oXOZY}R@EiN0>-H*%> zz`G?`V_mDEf4S==U*DFYTH7Y_7uJ%;u#{)zJbP*qWgfJtbzlki!Rw?(A}If*tF@&8!D|v!BMe$sXxl)*fl9xkoyr(;ltXg?|INdGQCGa#QV(erS6> z@Wc}*GuR)s)`$*ba69%#!(^z_kr*dHCJ_D4h5kPpuqSaXDHqb))01@|Ezt$UV{_ zF4-d;SaX)lC04X|_?nVE(qwq0tSRl(!S3I7k=i5WyY~;Qk+qBXkc1R|j6LKT;AeOT zeiA~9*(+T<#(?cQ;93J*9$;KPp~eqvYTrz3da~YZF!57#KXh^VfZqfB$w|Ot>HxBD zDtuG+9hz9z9%NnXV!f(){xVr7f3B3XvuA3j&H>#PYQ*+x*Z!fks8;i3*UB2#uJZn7 zQ%B-c5`gz`OxT_&uyxrpg{Fo7$~U|0nZon)8S7@=gP|?c_fBlpR>*#k$e(z_Q(_9w33|a6Y!mqT=Mdb zmuSn%x%wHoF@$UKzxvg|1#9lFf0wZ2rwe(lO4_FmbiUNkGN zrX(A`V3jB4z~pT5#JIV{Ph8=1BqcTB$hpqUhIwZW>4RNkDtvaaR8Z& zcN22YF1&1{J21?ly;x2jiQDPZeEKIo@ntt=*2p=09vkPyFz3b3wi4fnRdLL9d>foj z4P6vJKXjr=qrCp{;N+##6)%WfoevL_co~ON_TTy||L9$_GHV>P=^E+|=-KYT80=ev z@tr)ye6!|&ocZvlFQ%oJw%?7SD}?JmUo^h z-8%Og@OL$PNX7VM82*{<*q6yY{FH~e?c_TGV@_{5^OyKy6L0b0jWaK55?Iq|TEmp< zz*`}BlbFgr4&)3`LT0iN$U+jLlZ7RdfIxweQEVm&Y7$TsVzt^n3DT!#5jDkyh)P0S z0zvBlRSP~Ph)ZUaRv$o<)~A5B7L090Tx*|)04_5Lh*5zILg)KFpP30m2#bCFet*ns zW=bm%!x##jNaV2~3;Y+MQJfS`_nS z9``y0@2koQ>7$K41mZ^asFiuz#(b3+A&D2Ob?X2G!+Ahf%K979i?{ik@bISADzHR_l!0!NGyG(q1hWMT=#`1~e z6{h|8koGr(v@de>`?Q}tA016-06pYRuS8-i4U?bJ@A(nfSA?`Z5O(3E8gMp^ob$TP z2EGb`TWDFJQhWAchjziIlU2f6k%=Z`<3JuYnZU6+ok#Cv>~3R*>jv)T*~h$)xw)S? z`YE={KeNN3j9%i+zX#( zGX{$W{OC%uruQ%g^Z10dB13E7lkNcD2)_&8*D&r%_(bMeckW3VN?WPK7)_n090LCi z-QGI4T#Tk%Ix+q}qpar!)*~b9c^KQ6x}Ep`In+n?ToMC zk>5;jVSHW8XFFpo<+o|7+$$H~^EAE^$!})u@j9+E`zlwLyyFPjCP1mQaU1HFBpGH60LA(FXKEyWS zPq?rBnn;z@Svw@YWrU`-hz%#YymznfiO_c+a!zn3aXN1L8pqfz=z>p2;oo3<^8EXb z?=ARu+tTgiQfRnzC-EB#@SXh)zt-Exi%$CZ9rEHOBdqxPAIWaF`AbO~f)Cnnyi8;xo%)t^7NC>W@-?5B`Ip)b9fCU-AF3kosQ= z{lK}@-#Cx?PQ3!kpNq~K} zWezVyP72QV;eV1lpFhRNQi@;ZDE7(`=Csg1lXySQ1&$DW|No8aCHO7RiEDU#Ag(8T z!Cffeb_uvYC$6a<#C0y?lJTEwZvOm)f8NXfcJZ^^!9L3F%;XG87i)@LS-4}xuq#xb zm;2j2qH`%@PbRX8m?q8x8R$pu3^If3NY_hkYBe1 zuC3wfUfa!&J2|-=b-ga`?@q;|Vd) zdKq`=#iD6ZT`Mp^!MfHzpc5*7m@d+(cIap znOi=ChM&{tgMsnwqugHt=Bhv9l_@}{ z%A~&FX(`{&eOr-#1%1+v;Az($6`_yQs>SL4qxI<4LbERXU}j}#$ za>DJlnTcw+rO{Z|#P}nJso}^=+X|kwq(tRdwj1lj9?6ce+e|z&@vOugJ6Uw#s9~v- zB?eRMr>aSjIe#;%X%@{;cN6iJ61OUNvS{i0{}TR($^SxoU6s6JJ{dH_3zf7juo?Ic z^3z+?pJ7yAJ+SRU$A0F-<^NQx8;MVma;?0Ra#gA?&%_u6ZlTRKO$`s)eL*{~8UB)- zcj*nyYl>*ENSPd~KT)f*q3giskMdkE(;C5?GrN}apRwlnOQCInPx#+@Z}eQNd1QZ? zNXq4m8!|bPa(a2QvPoRF_1@ugtubk*mYWf+m+P-XS-nn-vNapxTTCWhH=Hx@?yP}$ zlRO9F+eid|e%vH>RzSyQ;A>`ViuiZY7yk)gw9#%e>y-iz1KXAOEI+;y!B>1gzU~U( z>sR3F0CUXOr0qJ>y`ge9M|L{+H2VDm+38BoCFt^Z9%m`jp;asIvajZz%Ab|hODx;# z>~|l5Cxl+hhA2nd5Vg52mV2G|CUDLy$(4-{PJZJ*($CkMk!9n-X%6jL-?2Nh@i7(< zUxn?Ew=7)l)gfm+JegfxnMPpGOihBcZaaUToV8-Vt~aIYD^WpAUG zx!%g}|IhejRMFY_WM0@wKH>NG=93?z(+%X470jix@ks!temNqW&K zd~%`i$(W*lpHCh=36stzHvqTr34M>C?;sBh#wWh(;1kLR%jkSUJu~$N^2xW9Jtv<~ zH)@9E1KkxESEHvLY$@@os7mdV^>g53FuoY~aUItw`eQOJM8H}~zFK~4;!H$_xTYs;6}E7oT;39zSNxWePDF%W=@}2d^xaR z#u#QWmSV!ZLzGM$u9hob7p~y2$59=^^&?{vO|Ag z8SBv46S^~iPr<`F=JhkF_G7ZP3q2*rZ6-48A#fsaO8X+0mg@hLTrt2cy7Sq5<3SwT zSX(avhxLN%%lx>$P1?av9>n$WemqY<1<$GV_k!m%Kb|?iJoB1Dr})SP*G2e_f2HF( zQO9*7|NXd5)NwtvAJ>w5hge?zPZwNAK{ImM-1$&(eJiqb>km|~oad=Brgg%5J=^)c zC^N!g;BLJ>@V+pH%Qa7`^R;ff~(g*I-RB;LN_^EHyBM7@5CP0>6|{)(Fn~t zd3Giq_?Gs?FCsk91$^j&o4-|Ri^PWopJ9*mJ^*htFh03+L-;}k`2rY)FOL66_$0~o zus-*MFV37xDW1Kfd%%6R;;h&t>{Rq;x#wHf>XEFwi{O9CC2==;PV0UqB{8e>O6n>);L{`%;)*J z@@J($*NdTRkmg0`Z9$p~KUBjHmtZRv^E?}xFNWqqN1?mKws>l8>PsS)y-NJ}>~FUP z=n@58OwgkS+Ehaup^IXlU1W4*{l$H9?!$_l&V)81mlbkY_~#;UoetgodVG>AhyQm8 zZ^2`J{y<0M*-gS9@R!JsQTz|m*NWcOX7bqFK*uy_x^4)$V(=l9M3K)L{;<|p_1SorE$`T8Zou{^89K;2ayhl4 zi2gs=8|7~re+X@T`-n9e63 z2cHM|XPf`|Zl3#^L}rAswuQ6C4PmW|V9jHXbul^EiVco<_~&kTBpF^wfsYO|&o?sX zwjw`@IP*0!-R``V=U--NPKn_*a})?XU*cO zzNpXCH8(>4nb2S6e-1iFg~y-2qKfhH?3ya(ha2C#*{-|`XcFPGX3NP`oV9j=cW~TB5S?mz!ACr(6#>c ze=odwVsR-nDSi{)n(`*Gn~*)*YkB*^4P_Gnl}u06oxm<`ALxG17yjMgyzN} z(0n-U#M52^?IzNG5;PxyFFxFH92>NcbwTccy#yU%A>+3})2g8<0eae8ci@{@8bZ%Y z7-!{(lorL_NY}`QOOwhAok4m{;av^B>yk^=@EUL!%a|;~R9y*YgsV2$y+^KNpTEN3 zlsq3L$nTxtzqYBkzSgLw$bNowOjOGXWbvv8i|flSwXa-&4|^kJbRNXN<9 z>7S+jP#C|#S1kPttO8#Kd^n7KAi1mWJL)d@Z_a(ktIglttU23&LvZCpt{k}z9kimz z`Q6Qh&Xu(3!T)ImK8fA-^G>qPJ4eg051`pW{LykI$dah)GO=OZMpMpfFWbF88Ll>u zqn@j=xc%jBz$>d=qkAi2K+3A_9yx%ha&00c>^i=>Z_K_EP zEpnim{j7rVs<#b4<8t~thE01gS9ym4kNjWGdBc_-+S>%)UX5~kz?+9LkEES@7+1A1 zwME)9;J=i+ijT5q-$Q+|dA>15=6J@k=E`~mT#33I$wNnlhRBDLawNs|6ziZHIg(gE zrBB6-)OD4^xpwb*o~`6v13HDskk*GU*(f%E$dF%39oCoiO^bDXLhjNKoknzo2YL5_ z=ri{~4O8 zvkM#PC-gZt^clSV9?!1g8G7|BcKNJ4r8t&1A)ACAxJhb`SF5 zCDXhZU3QyBkvk4qX*tf?V`9Hn&Oo=uIEGMObZfnS`f$hgVff^wkL&Xt2kBepoZKnm zjwp7z@!4$WquaCao#@zOnLna;{5P`l+u46aF5=fL#;1uayaSzdHM+XkYO|qHx~8TE zf6I3v&*WR<4714b1loe`-gy!?M?aP9pGhCF@_dMTS#*gibQJW1l_ssakG+jv^xN{S zHskxc{TJbH=fvVdXz_X*kx|6>_yO zm4n~|`kcf^pzS}-6&kVkHVBO>i&&RGJWp5ybPB@PHVCZe1y`V*Cj)S04F=cK%5mE) zwR#>rr1JvjcZ3&OsB>PtfQ~0T6@>M^L0}!&_8oz?&xzMS+f@PhzY&@Y;QdPAMZeuF zyo_yXo5yc~|3>NvUneo<^BTtho}M{i90PsgGVku3gU=WJy^20XpI2JJ2aWgx1dmI> zr#!Riu*bVh(M5eSuS5A6Salxqz{>*Twn6wTBXr&&fBd*EV%`-Z1K()N(BVA!U*@0i zt*m#FvrT?yg!^I2WN%eqN;nUg{uF@eoHPo+@egWe!F6SY=(@DeTH|124>>ymP7Y~ zVXAj(tVZ7eqZ(P zjvVgTv%!Dj@81vpj1c%Q{CDAJPMt!7&+plt8I#yK z=omh`XLm`Asq<`Oc>8;f;j7&zX>gF=XW}E-(;CP>l83>3;-1~I7b5T$snTQ1n6JWb zLer0FPxx;~kcaR`oDKH>3V~gngMWg2v+XP}pOyD7KTp1)+`0JXwh*|_hwoQ~;QXJT+-2LJA9K%f^W2iXG7zX7V^5&XPEIuRO%%r_j_(t;ho<;5q4T1f9 z`Q`^_f%&X_!+Lwld=;4~@SRC!o&)x-zkNTM6BdGZ?JQ#$6z|ym<`egx5$^}z9f0?9 z$orQ=U_W2HzegWuk}+q+J8S4Ec;|e~8P_V#z@BTZS{efT`OMkpfIBF2HvGF|z)y5u zYZc|rC3EI{Px$W&fq&sy;2)H4j@%F5+~?<;qf1Vc^A)sr%6vV1fB$?fLEi|~6AJmC z6$1PD^3CM}v+?ZtB7Kv?+I_{UBP@YE8GK=K?}XTg;-l~_F*{#~;>^t%7JAB^B@%WnYukXLyzNjlp)Au1HhUI2Wt@dn~;q1mY zP|!h68vLoX`zsbLiBWYg;$K`soZB|ynU|PVT{?a)c`otIqIbQB&mi&w`^qSMRpQr? zXL8?>Nn5d|AV&385i1~feUJt1&>8yuK@CT*is~o2un6pHSu1(XPT62T8 zW{pkz^_ur6CvCmW`_tMdc6iYCIm7VgS?Zaz$4`OjN5J%ZDKEAz_l{N+62rm0wPwxU z;Q{aFN%nRRG%(lM`#j*zvbOF#wqVw z`Ia1udOdk2^#bn%7f)P{d-9C-s{ama@7_@8tj^)g&d)XHy)_1>C#%?b*BGM{8=qKH?3b^@ z`tTulRQ-o$JKK2@(a-Hhd=`}nj*eaMEWVGk?fD9S=HZLa9x6Pmxjsy?=O^RS%d1q* zCwO+4xp#E7*bRv;nL7vhojyWsmNUbgC75%xNpl|bt($`n<*|(PM28RF&5tAQhxZbn z5J{O^@jrFeROMHkP<3KY34gkYf4n8dzH&vzTWiS8>FQMZd1V)mOt$Xv4MT~3|Z4V#4o>u{_ODW82XEfk~~`c2fiy~)Gq0} zHbyyGXk&$Dw=GLanH;UR5p`M{!b6F)5j|95lTAI_*blvanA)|2Hj;)Z$Ky9aU)r!R zADelXZ8PNve+FUL4sN=zWxUX#9lvf@yu|OPxJ;&zEoqvn=S)q9Ib78XKZxA;4jz-3 zAPaXO=rkZ-SVe@YBcD}yR}*(?5HBs~X{zue|tcPOweOFHr5X6N_^u7?43P+ zKjWk?5Bn$5#*#670)1A8^eJt&p_|J%HDq=XZTtxS3dSTJx$biOho_%K6P}XxC8kb( z3w}AL;*e*i&}Yz1o{6tW;Omz86M&~C2^_?#j?>`z4!f3lix2RzqHPHdJN3l3`zE%f z%$31lw(9dE-nCiQh&L4R$tU9(OQxIVR!>*MiTAyx;#aE91Z~U)^_4Xv&B3Ne+_?Ih zMYZN=a&B8b$YmaiAJ0RssOr)7MJD(+H(fb7Q`b|JqRKyJ9A#Zia|*P`+E&`h`PBbA z0~v40P<7kK!c)Xci~Mquo7kGc{mATJ=iu)&UtqUIrr2#!W_x{2VPuX8nQ8jpMY~Pw zBVD-}J6264=dg96-T5N1%KAAz#+}VRhUC6{QH$;rAO85>ivBsgl{wrtT9yBnIs7-~ zFu5&Uf1Q+{C*@=w>-!&!WgheRBj9evKF7~!GbX5dr}UL=cUG{@c(U+6A}7k2Yq`)X zH(NQ$$6&K+_WIny7+1}SlHD)jCwM+shc(I?sFQn{y}gY2Wu;EdeWtk!?u(l%WuKy~ z;L^|8=hBxj3Hop&7Ii5<;bzz&ORI6)DSLHqU z52XJEfqHLKM|>U+2rT35&Y8$T;XmT4;k$-+$0geHW#7ppd<2(7!3_ma;yG0+CF+?%VR7wb)bec9_0 zc@f2)z)o~B;B^kAyw0mKFUj|GCVU9r{Y!Ai99jWBLir+>ewYuttP_=U7O)VnlaLGl zcc-Y$(cn#RC+${gkuA4Y-;%#Iuukq_j#a@2e`k&z%J}OVk7jm#c|m^O#=!il;Czm( zliBx*j?sWh9)K16RI}G6JUQ4LSwUaB1F(+6-*+ZhWlTFMr_0qiRc}e-)2iCd;HRDE z!fWd#{v#lB%*?g#L*H8hZR};O4Yt9ZSKq6RpVNk3mob}ZCkmdh!V}rd@lgJ`g|R$N zS+@~c23?8$@{bFdWSP$A9ks~vn15_Cr{s6=x$v+j-riolQnh;`>>bst8+?0ZEs^zO z2fQL{sO0ww!s0~FJ`*gnfMp?NeMVL9%Tn#`c>AIn=obl&O~^+}G`cJ}t{4D^eiq;{ z;1qaYQtdijfd3MIxznCY8&~=J>F|wIHtdgj#@5jWQ~PTAa(?d^KMv|$SLSw#8Z z7?v@g&NPO%8AE7UD?0I+%FaRtPNUsw#>kvH1>d82PJWQi&^aqIMtCoqcLRA(=BDT% za)v_M6CZjFbN+zQ>S`wTLiCax=o?D=1GFW%?yFeOM8EVH$GM2#Xb~RS$z0H38UJvxED?H9h0=bzaovj+~L-$GOKc&D=>lEz22~IL(e=)9Fo{3Syy0hQcm<9u{S%}dzG{0 z+vGkAVmd9@Fk%Y@<1$OJJqwUkd$BRQh@tG_+?$*M?m>@|GL`HP%lr`g*765zTB%F? ztUO=D^TFa&f-x$(Ez^+bcnF()9CF4PSVKj&iOdunJz8CRip+G=eo$t*HIwTiqhDsS zHtYA>h|J7EW(sdf420yTk>3(?*{ntBJko8AO5#7ZS!XuF(JR&yNW7Lab z-+hu6)tN&a+<18SB4oatNvvcZA!FYiN;@$Ia;H+3d%tYHfSk7s!5#qSVX5|&lQRFl zCdUxzddHBTcL+HzFiRVvM~x4_+z%%@ua*9+tjlMDQ+O)K%RxA!BmHoWkL2EiDWY#l z-m9n{p*=X&;f#$XP6IyV&S^(35`)s2#CY`ua z@IO%RF1ewWJ#?AC%2in|t7qui)u0?Bi>o>%I}^IC*ne%ebDtdcpX6Qg>~ET5;k?L; z;NJ?Y@;jZF=F-6L?N`A+C4Sv88Xhtu6UOuH(jq%Wp2a|ubpc-53QbH`1a-%^b-jGS zf8^&$xfRevWJyJUCRXyan6V+GzlKu!4*V{hN#EEl1+4Flv{ivUvGd0~`yTehWZJGN zI)9j}3QvW3+^cM=xyL@m$WNo#!H*%89OKW+m!WA?2ll- zVt+{ASyB$1S+D+nI1^aJ)~FU6h5m-oPFeo>kIl$psJm1vtOutZ2}V%Ncgk zj?3U)9ar+i^#E#p~+SX3pQv8-iZ0A_$$^K!2Yux$cybT=s?KJ;fJ6JY2&c!xc zPP;OH`}sD6zm^2-#T@u&6)^en8dtX$9ty>475{a-I@0}kt*>1a#H-NMT6AWbmeKaT zwC%D$Q~EB2x36TpGM}s$oq0ah0Q0Sar!+OZKzz*O@rNR3x{wV*$3l@2(grdlin&OB zgk5&Z%QG|QnES^&V4ez(T`%xt`|U^JGr><3@+}5i@)Vf{Z~soWgWg{Ar55XYWO{y{ z$TL}EDl}|O*2kw9%Uc(oKkasKcWfx_#?$6ta&iytGN#`B#B#|qA4Pm} zH1jrwd243givJ>qG4!xj;D<{H%F)6xeqLWT;rsDAdUbyr^YFLGe5nTi-$xb`bEwaE zGy4R(TpL7gb;BEnrJS_SJ=Wi+{m{APqMiM;^9}P+_GE-V_G0&U8Hv$gUPkb)jQx=_ z&4V&GwB&So>|tpe@wQ^B|9`$JW>J&ppN82mfyaL+F@u!C?-4Hqv$( zKCn#WtlnQMI#x1~?c~`4Wmyf^Ty}o_|lZ;FJb7xvVm($nJDc>E?%>|Zo z=_qdS6*~8>95wjdvw~xZarZEvL`N6>kuzJJG6x6KAJGrJQ-CE0zeXrdqC#Y?z5M*s9z;fT_^s%@5^U8 zemJ9bCj69vpZ)Zw&xwE!WH3CIF?R817u{=`E3zSOXvmrkZ^)W0ahjpHe~UW%S+_d_ zI9V3($1Dr@W18{DEDQK!nz1XzcO`Im(5ue`?_0q8D&?Z^5f3KE<*sL;#cwH-P0WkX zOV$~Ee8738cDsRfJ>`P)Jm^ajzY>1`ep-ty(!_sqr3HKw)_&iF&fhQk?Hx0|h+k2z z8{QAaNl-U^=nU_K*O&0l%sSO$yvQZ~1X(l1#}Yc;Ztx)Eyo0j+Fo*cS{^JboyJ-Jr z-bDew*s9`(d4xW;vOeZwTka>vZkZO_xm@}MhTu4LojFwBh4-&LLm%C|6F=5qV~+&x zE6&#cXXn>{B;%NRhCakcb`kGH|H+|0@j1PPd=Z%4SgPZ~TFRK$CDY@`HDa%usg|dXPP01?>~9(_c}IqJH4TqbtKW@#VExyRY1_UY zhMh3UFxnBu_XAgtB1dPq>rQO z#``UPviamL{fzG=#I0tK$Lo*4@EP(;`dmT2Wpd)z&0-u|V~Ncmx6sz%*w*NuzG&7P z&H`E;PVJgYy{(zZYAHkAdAy5YpFsK{j%3bzT14j-`hNB6iu^tQ^6#hn@E>Kf zAJT{aC^_qcHSx&EX_5JPoDp+;aG@Hun(uY&k&6#$leC|M-<~z#dCm%LBA<>vCZ1ox ze%3dP^)7rI-!jh0S1afHscLwVmY?@tGJZsp+O-M4p*(*j%x-%>HD&T$=KN`r*Wxq! z{DSeu<5RflXgAQ)8h3CkgtuKuEz#hb=QT9b&0ISekcu8m@ zbeYE;RM#`khly*DydN^|O;vZ~f1{}>f+P7)8!ykpwiW+86u1AwB3AOz0487l-HkjxqT8DO_@c>A!{VWenmc zzXP6J2TwN9_d zN&O?lGl}jM?CS{r_xCSU!xtJ7Tl(?4&sQFg4w#Lf)=F-?o8WhSEm7<*geh+n^+l(l zUD@NPx1_7~aGs0&B$towYc1g}2;SSion4GvDMG#!a(?JfZQ|qO9@x&Q0l%Ttc~(WV zjK0_J$2|y)Qjaq&wqV;av@Pd#==0qn7uYuqNg&4||4ocl;1YaT;;sgB$PVlo6Y(OOHz07ll?(_q}0Avu0nhbb#QABkI%F>R&nws18~&LlJ& zz)uG4LzivDib-D$e2Z=!gt3u(B?ZQvj8FD%gU`2Pf5~$X&-;WYi2YiD9JUZc)*#>f zUytmTGVTldUg@SxC|>_-5SXoLD$m1N{ZANGf(#RcHUHk0)Sv;Abyd>Y0FJVh9}DWxrbR2pqQ$ zfaAhJ+AEEwy+V7h+@o#6-rPZZX(8X>TVvu7u7@vBLzuX4*3i z7}s%iIzF(C{rHXW*DC=Z$Jjp%#Ur-jX?QFh;;$!nLg@IQ-hl!2x(2DYEX-f8NMsst zW&`K{4XF3-AoX?{RGyPQWS`f9jw`WF7G#IWv}}{TW4JuW<}PG?5xm9nzkkk)OkvKa zf5e=OSOqwUDD>}=#CISz6)lb%bE3u4y;xa9+bAo~Uf+)*=VJCZVj<@+;$6n9Yz+O** z=zY*K6TPn}{jY0El*!c_k)J1an8b*%r{!R8QP=x!0WLRpYK#Xqkw4QKIFH7d$j`a@ z@E+ytdKf*PxWDbJ--+l;dFVo|hCS1}hWcX&O)=y`DYbVXXB~U9ICJ%ea%6eg7v>zW z><7y_aQG9=xrH;7oPpl8hqH1T`ljR{K8!8DBgfv+rqr|@$mKSJsng0>c0O%z_7KY` z^b-30>LTTAIpW(=5U0Ea<2cigE+Bf3lodaaJd<~_zn;$XEyVpzCDy9vTi=#%;D7SS zc8bl}%ow7v%l^(;x^>#no}1A5cW}NgHWplF#I7~D3T>`X5;S`jJ9BFo zb^`c&_Ly&q_`iDbr~d7zZ}Xv;VJ(OLtH{ef%r4F(j*vdu`8J6@z#I`9L)O!DnKR5C z{P6nR*~_=a+(`$Q!soJ4j5(MyP18A;6RCgN`>Gpf<#8v3u2-Oa~Kc0YLh>fO)Xd;3Zo^;)QRz!*VX zMMTe1+6~v+X_zwv*u-|1@j00**^|_+VLwogsEh2j=p4I^`@QO4ER4>H8Dj6)NIc|= z>!V$_WW2p9iuoB!eAOGoG-Pv+bT)As>yRzTrt)U?H@;&3IA)A}(eKdJeC&y2A`4_p zM&^O^WySVcYGt1|M>(>`*liOs?Y68~d;M+pp*fQ>?d_BPZ_(}vPY-ns%lP}MY~X{( zbXbEvywC9AO< zuZ==)K$h0I!Q%w#9m?2|`bH-HO3hef=Na)0?-nCB|LX?FH7?be?aR zBfm%TK0DTK>xT~-1>t)~hcB*1=E3AhyX}|opwOw9|32tc%~+e@iA;E60z5Gho)`sB zJZ6u}nFLP=ohCjV=jzRPXO-}0C>(2n0owPL(pD3DWLv$JJ?e+l*wp?J|?S{uz5%&Sm7reD#{ zcFmqnoqXyf+v_h!J`^GQrS6{t^*+t`)2iPA-wJ_+zN+Dw`SdlLzOJUP8T5rLYZrVy zL!HN3<6K{XtJ|ovBv5B3@SiVySu&3VKEBTvz6|Ihuq{K*oDXcj1UAVz5#%qy_iW^e z@X}Y)^Vc3+aOc*HK| zJok4<{#O5ap<^539eu@-j*@ip=-+Gi=3!6QFb7+fa!=h}gJbsSqWZ5m8;1?Gi~X^c zFR`BNdQjGrVLiR9CzCSQ-4K_lcCEmtE%|OHVW*3(AnQq;QluYJ2?yc7PU@A##I%$> z!X2xD_2h|0;(N-px_OMfSwpvBTqe!hx8pZ`uRhLy_Ei!tLD$LT{oUXzo&U{_iuPsO zE80spncLfpN$paO7%$EOm|BiB8NA;;Y|zhZ`?6x4zK1yjwPdK%&YmE$=UqK_K7G8! z`H+HuUQ;1!Gd9>v_IqnHcCNWw8{v9O5&w)%V`4mVUPJbKzhWF8E+meZa~}o3WM8Dp z<-A4>XFqoGZ6zM3Hhi1y7S4Uts=jF@$n7NR)CSIYRMz{;*Af?E)wbEjakgWblw+?; za<+-TqMEZD(zd#XeKO8&ERnMt{MOGLaHgXKd9uP5m9y?r^^4X!cz%c4oP%E&ICps% zyPL6Bum^e*@49!Zjsu^trroG{eSa`?_~!H7$#*T^pYXkq?;gJA@;#H!Y(6g-I^uz& zwaK^(9m(;HiJWG}Ir#;pT5scgWO>t^zGUQP3j5D#Cn|TBE~#9(^N03LH3e~VZy;W{ zt#+cb3tn1p4BPe5f2h?x4;Oj6n~J=(#57d{qZ@x|bmF%B$HCFtT3=qZJO@wU$}tPv zJ+7a3>C1D@L8oH;>lpuzN^+Hfvug14!Nt)zdw!^Xp_t>#u2Gx2*+ZDc_;YEyoALKB z{qQfujB7Cy$#tJG%V&2~l7w!HTU-vr{NvVW?|7hht^ zxwk+KU-z(Ip58lGO%Wd2#vF5AsW!`*t4`LbOXxeucQy1qlfExmpf+D3^r!FOyD0ju zq3@aWeaS7nr|%^C4!#feUGih@ydqYs0vcyjadvBFp|@{#q4(I_LT@*?wKu*#y|D3( z=|zoyn!YzGZewZVWqqD_&0EO5czcaQH|qKrZR|D9-q`(yiSkQ{qr7^fojW><8XuVM zi7zByV0d3)W8-x9NX^Sc_8!a9yzb&K^lXzu&rkP>y<^{Qdxsqw2IWH!?+?5YN8ppU(UnB5!E#wZ2#s_&ni)X;NSo0p;A7yj*%;;!#sP+=}2Aa3i zPZKcrkPESKNMECjvB~Jz3w)hF#V=E#rmgMr;kstdGT-G)&Ig8(Eh0mFH;37L1%`U} zZP+WlL)+c8opfauI$MVnZtQwEd80EEU(Q<1+fp>cRtk*^(2qT1*QK^{&qUYOhD*Bz?uRvR z>lpiqu65dpNHyldS;TU!W6$&e^0e%NILRJS4Dhc-hfY7H;|eN>jc{Q$gN z`#-nuuGt>x%FZxxrYq8Q3uWII8J?>)A8nfJ%!2Q}n@^r9Czyp%nQT}Q#eR_sUnSmH*+2Az)$dSjut-v*`bUU0FWdP@&ugN`iJZBYAYRo>9F zZMtV{q4y~89R|LxX^odQG&Sh+$}?8;W=qWHPOa`r^fK8|5zk_i83OL=sX(`1A!E^UtfEjwekKC~os=6zsjmyx_r0xRZ+QbH+kM-V@ zZ9EWcqmI1YeStRM4bgLhe${Q%>-Vc-69xUM^UxiGu>2y}&Uk!~oPE_{fxrG+U_myg zAI9GSOxyVm*7;AVGxmBX&wJgsNzN_K_THwvA0(|?^9=QZ>XAK#1lhQD+kLN<7XpMuvea9NpHF^7`cH`3N`)3w;_kA(T*2VndZWdzS6I`3Z zh)IYq?C`vBjnks3y6@tbt@)05-Cb<(zG75)-8U+)?8)}sO*<242R=Kc9rQS9XMM1p z5B=@*(M~8V!fSG#R%FHzrScxnoI$=j?)xRKEtCB$@hSDvkKp1M<95FQzyFUv52^dZ zLjM`cPYV0bP=*&m@BM{daI?AV3+4^Eb2Rit$xHMvJMRt!>U?vuPMOo;=rxnC zM&q7ksng55W5}z%-)VX*xo0-=bQ3b`RpwX3?2YLWVH@4XLh}7b_O%+{o1WgZe7buA zbL2zj;3t~5%{aF&+ZfBccc-TtADMn+kLLAcF~&o*@eQ9`${v++*Yst#a9^A8o$2Yt z%=xjGlYPH+xN(o*dnVr_(Gxb5IXxEi%q*UbHF%E}53{+Sw|BU& z7-sWi&2qY*H`TlE=Koxt=U#2?zK-u3<@aoBcLm?``7Gr9&3s={R-bLg-!SsBj%-bJ zxNDjB__yq=DcHT6Et8nP%=6|kh0gBTMfm+9dx&=__w1iVOfh%eR^aymZ#DQ1$S+)O zhOV;Tk3H(BSrfPeZf>VH5^P4v>ysOiVyQAwCKCzaU&qvLoZnz+w_!ir3IX zS$A_;cjx-oTk)@0@vlrnpPWw2z!Y@Ki_s}1M%2x?+^lD^o{MkHkk6R@q*lv5sn}Bo zIfE#1mLW}7sW}kgyQ&Qhn{4^fqyXb>&ILK7afwXLd zha~4-I(bxYIKW&y#~hEA`Fe&q9)5;7?khO+9Pi^Dwx}aCS5_Z6LywU&^!ROW>ie(l z@pJP3kTqw~YsGWge~;ZEd?Y+0cqSHNL@1qiGR`V&5L08C%Vb%!DH;A3IqgO!xzRV; zL^h#6YviX98S@eS_x+Nb%oa8L2xpRlI_`bk2PLs<4eXH=;Cq^f-$LgP;(E?@ra7B4 z&0>4I(D_A=vocpCHdLn_HpO&&Sw7+{Owc|DSj#d#Ozq4RK8mWTWS?d$^C<&gNhmLi zZ6NerM!U!h@6ip#HqWZ7+MVd7ovccQPFS4Che&uaG!HFmTbKb)?bnx|WVAw9-)mD6~ApXj7+bkd_)(KgQ$n)gVH z=6xLeC}e^Lu2%4STZY&>?Bzau-9~IL^xW(D);1P_FR@MKjuP>mkD&Z0zD3WL9HXOo ze;v=#`L^Qklz66(*^?okM^7tzCGfj#2;-|@{8s<}4g>%7HnE-JXzM%9iaBYcS@4Uj zaWUtjxa+8ge6UXeqr@Q`!RCmij}IxA7=#J@_3Qi74D~*X))6jc7`yDH3;&#y5lODw zdG;;5Kz`g2l0QLY8L`DW{msa<68sBeZ)E`aA`ldiWMz?WWE$d^a-h-ooCMaUj2RJ78dau}A)w zdIO(JU3vEb@36^xUjQD_iNq%ES;C#H&`R2kVy^ToRvj&2YDxtH@prWk+>j<5nRUq z9_|+j#!@=z>mG6=JB`=%b))ylI=CHqE_r7&Brl3R$WG#0SXw zon2_kk+q!P?b+x;3Di$LasBQi;NU3ovzR)90~xpO&s@yC0D*Q`=UoGFFZ~G546GwI z5qofTPa}60!8==**qq1U2M1-vmKOL!`v{hY$NTYS7GB8p|MeOj9++R9(?W@ zduCpc^=pT$!FNNa5MBSsrRWHl{lJ84}i>ooCwGn~QvEMM|oaCQXx3N56KR`@#9 z#`sLyU>*e9pv*M+FKu)|U&-_GPs+$#eg*j-B%i}ri~|e3Ov~LaQVi4zr1#%5nq-8pO%JiO5w-q zh8OjD2Mw+BG>2y>^&_|&lD=fUuEh2V!p8HHJi2Oz^8`BD@v(l~d$J79b#ktmvrgi( zQTWp|@uxkUuu0W%RDr~W8KiXhxa}3Ob@?ntLNwU zAoJ!E%c%d?j4ex$Wf{ml*<%s9W0tFgHdzN9&a6CV7CB>rc-lvq$G-wMBBRL9 z8l1m^A3J*0srVTdf}i21;Ab!LvmZZdgyYxXN2f6{3Z55=oF;g-WVC6mC*JN&8+}sV zbUi$0td%$H%< zz#sE|^>m*v#p&yWH}R?ZjABO*-RQ;!G-Z^wJNI9^$a6^~HfVT<%n`ZgJZKLzHu%af zIPT{Y8T(-JMshyMI0Q!1Uf&kcUC1-x_3=#Z@076(R##->DdXGY+j4?6RpLd&o)h1p zFDp#<;mNZp{1^LGev7;cmXWnd?5kisoj(VWHJ?mBDQlXMH6_}qvSv5$CC)S0m-vi> zZHnxv;*Nx%oRRt_{GS8us&wRtk4B=YOM) z=#qM01KQEF-0GVA z+6Xm8=A6jUM&!pN#%2nfpAr9FuzhQ~v|-eJDAK0Lmzinym1Xxut(CT)W*rm%mx=hX z;CTr-LnCdNpa%&&4SWaT5!)mb9`VEY>y+qqvcQYfA&#xT&P?LM zL+b?L7}&SqEfih1G#?DoG z{4sjM?Q0s&P$=G zF+65yDfwnJ;IRr{{1Bf!x=pSi{@Gt-t&5Jo)a^F4`UMkTZT;f z4F21~*_f9kKL`Grn}{iX+0rm)(wKKrhY@GYSa-1w>El|>xZdmP`Z{z>8e=*e9Q%mr z-^EHh5Hg*-=tl?X?I%X zrd4TGn--?szA5fP`^w1|F4}avA~DBNUphNokbW}B^1SHpL0s{UuR zBlMWh{=n2Q=m8Fd9wq2e8^FQ*QM*$2M}Y(ABJ>HRjh+1)soR9ynM(Z12I>+cQ&)xm z>>27lFzSueut@3>-!hu{F#q-o<~EIaL-MWG+3~eW-3{zJO@+Uok@Gy%ZN>-wEOnn9 z^{3R)#1dZ%PmhMDAE537WB#tF_C3Pz~uS4C3$NXgt zxmoL!`0Rk$h|hLvvb|%Ij2B$FnS1`ayHg)TKe+aG>fX-TE%nrM} z5i>>It@LX`2ED>~KN$1p)K}P-DQZx=G8%}_q+dA?*%p|CiO7kg=t+A2UlOy)+}Xg~ zSt785Z^5tNITXhtrv~H8p9E3+_YlzJ5R(mjvs7;jjNZ^?gzwItbnc|Dm{V45>dZ2tWQ=>6<+?oes^= zLGUj455@hwK-~)i^^BkT>usUlR_M2b_kwf5{j<<)^_V}e`5IjB2;e>(+&{_x64qOb z_=o8yn|LYc+FOeJu;L?(%s6)rOz{_+kUhckN3xFT?XVtu;DbokKr6nkY|c)W5|Pkb4k4PopO%cyHTaims);qo*4V2b0P~fs;|uUFj>BIjZROz4 zD@m!}R1;{gleO#sxb_5cUa_wApT#lYs?E1tKsy%% z+DQqt_Bydny&#^4I<;L?~a<$f?|1IF0fbfmXudNbK$GiyU6Wz&PObj{Eq+A&> zb>yaWtiyji58qNSmn1ye&lf>n4)A&Z^Qa(PoxP+9Fi|)U_0oDb*;>A(XsbZKU>ZmqW=~1`+VlR zmH9rfP3gM~UNtih(fjqhPJ@+^eFQVGn?ufA4=iT@7lJPtZ>Ve_$78>2AXhy3R2eVr zRnuM%GN6h!XECR8;X~pzCN)7f3wyT`BX6QTu@OUc6p4!$eYFH%CA{Gkzppu6*>X-S z+8sGv*{mn3cAMWp?qD;C4`&&0NsQhKd{7c6ADBa4vCq~aGnaw)S(E6061)lwIpgiN z@m9O-qG9&>zZPB!==uX!t1kp7uar z4cc6VY?5(h@q3ThMf5kDxv_w`aVzuU_6)nNE{42qMR7T`8TNMB!>L^#=lWYl=c*JL zdq~bq!LRTs{HdR}6n?A{9(;x~1I2-yivn;IB4tpFz7B>79DKdx0|6D4zKkXU0h!_Rou9v@g1P0Jo>C z3&>2b>~%%a-tE9MgfYE@jJ_Hhq8XpOz$bn{_StNV#};~)btlii0)HEk?Zg{RGJ*S{ zwBc@SPQ~XLbz8k)LZSgenAPAShB5~&% zu^(cXtA=dN8Os@$s8(e&gTv@@RsUPwb8k60z1cGtQN@qj#>kMKg6~Pq90vnC#w{wAju`@7QgUlcET?iowMwJ}(nf z(?{&s5p;6NL+>L#p0%vpN9_A8=yyJ1;VapjPk}D-tnhkOUe21Zj&g64%VeFzI8P)7 zg7^X(W!7O2Wn9PniGZ)koAm|f$6w*BLtdJ}**@I9=ss}&BkJ#={w(OYN9v=4?h#wJ zoc=gZUU`kg5daTy1Z;Sd7ax0<_Sp9?m-G6s5T~!VH^g55pR~6J80GB#9&DBMb7+IO z^is+mO{Xk-m3cogIN5jZ-G3#zC3}{yvS-=Op5=(*5jpMbS&nEL;X2BhgAX%5T=fOI z-YfWm^!9&fw{=`>xBWiVUjO6bl$>`^RhP@$0Dc``z%S#_@x<6P z`hSxAfwHea+`_vl;QIH5q?Q}le=P9%*9gfQUv;@Ezj35}<>IW5*WAth6TeT(&$C(7 zFuBvxVocI=P);On(8t+>2KLSM_R-O_S*rX7@K`aFSPIT)N?Q_(7o?Pu@M{yFjZgd%ywl6Ghua4YbeaqW z&&AXU;#p`+oJb<$n9A6F26AXlQsw#BZQpWd@Znu>VrF>*Qt+pEZezij^Ys(%3Q(BAX3C%p4YpuG;- zn>L`m4{7fy_(j^2xSr7V8p#Q$!#9Dk2H}&jrmNm8@cA6=>T{EJj}#^4F!uH%OOsq5 z$h=JdVAU5h7}MqcF+Eqrcw}zAYhRRKoSd^|WpGT%uJ_a5Uv&f+zA%pH6dsnGI9c@l z#Qlb~iG~q9#Z&RgXd_y#XYN>-JG1a@K!Z9vXCExgnR%4m3U65{_oTsIFL7bdF>gO6 z2S=Yu>Kwv31{-i6W1dtnKU*g=C*fs@wURyT=L~5r|3{vTB>`UEL)#_eRQa3m@|)?O ztoc}_xPCJ{KW{O7{1p4N6~;7uj{g+8w$o2CF$rG!xtMdnem+vkt_uh0XA=F$cs^o0($5&_CuBSdSfw9{3;9kT&q#ed>q5taobiw6pKGL_ z-!M0yVmz|%Twxrkk7oq^Jef{Dm;v%c_BK8u7oy1Z=b87qJQ`xxeTC4Y*8?3wd7d?+ z{VVFoJ2UV4m=Di09|Cf${UdS@3jQU&;a4iT^9cKBpCC&r0`M*Y|2xPZ@N*si|IqRO zo$||)8-SVEXNh~(WyxNhZxVIB8EqWVa;Fx){4?TiZsK`9xoAE&sa{<+g0F^5<|lB6 zu5YXlw(BS>usZS6eIA^x*%C*OC&_csF5?`Zc3_UIwn{Q}!G*zN|_{b0LqkS}2Z z?f#hgRKR>XfNog;&bOgM6v!GxS$#f99pd_C)9!PeLGZ&nvLD_)_MF!lqI<-4`W8D` z;&L7`jA;2#MJ<1b`%T{GTkh(X`@4OVtE8OIkl3?NcQ*7Em_>c`0?PksqJ4$8eZEhNJ8;2#pT0k8lVb`p|l*`}bA)*?HjVVy_z9+QMi{ z;Hvij@A%x1qK{8>O@V(Kd*I-zzg(&Re@Ds^kzpx4SwjWqDJ{YEWt`Y_f^+mKS@#Rc zy&yPWM!Cb_TwoWOEax#rwiohTU|&XjUJ!OKuum;M754Xm9o-n5ru3wvd(c)tOl(NX z{|=$aA=dUaz%KEbAM+8L=M_UlOCEZy>>ujC?=?8(eLcGKA?P)c=kj05NS#CYgL8TI zg-UK2drH{|*Jz$g9nqoF`Q3*6n!vuV;8pCAL+Faxn#CoyN)U$Rkl%-hFGvXe56lu% zC+!TSoe3fRMe;lwcuYT0&P4PU>7(QV_KVPM#rHD;TQ3FuS;}Nn&x>C<^qZA%#o&rdAAoIRe)*vptfejVV6dAU10maOVPx@Y~6E@thNma+zCC!Yun4`|!h? z9Z^=xv5qh!Il~%~9YgqjQ>8eJ_$NQhQp6~m z^gHcuykD)B{0_nM&{gb%mMT+^oYO8i$b4kq;cWKJi6+f4O_^GfdzE7mxO3lSueUT> zb)Ul#V;Fb5g$K@U8eGSWLtNi+P9H{db-k_}h1h3Bd=8eX)%ZB_kSk6%bB@^kx<>ky zb6w^%d%kr$e*MwdX~+V(hj%Y^_CbUF@KSD>THQ+As*KwMuH{bNAAv)+#(m};KXf?3 zDfX_j6q;iT`~5_kddGB`<`6$i=sTf)cvbeO+oMZj# z%xelB39Bn)4(nr>esTjHi@TOkL6`2yu$Yw=0$;4Im4%HVh3EbrS4oQJ@VSil@>*|YAII04xUT~66_`1vz_E4A%X^nrYZ*#PLC1)}^ z$y3&1Z6r1k*=CQ~Gd*LNedB2Jwo8rlnaQV-Pb;4pX&bH@oz{BQ18L7*^}sWuZe0Q2 zy=_!mteWO1<0G{2-~*$-U6FhW@QIZGXS#M0{wV6y!p{!`up;)$6#WOs zYNo7}vJKqnW`(D%!qeu`7Rd=PAHJ^&@VyGbMYb7S(6{79sG^R@|02qk1j>fa4=F2W z#zN?>DoIUZ+Fa14uThDu9dS5DPP1R-YQ?IDi8hmDehxSkQ=bmE! zahnbal4-@+-!betejhNGk3kb+C_SR2{L$Q*7#7yp_{`c%@vhUv! z{9l!(eqjd}%-6a`^u3&9V*7}H%f>$?{z#D@X81E6oaDrDuNSeb+0p)Y^MoIGC-Yjq zqqzTc)``O1k7(wW-)2Ou{o7u{FP)y)|SFVgFgz z6@)fE&YmB?&4iv{Mqh|TZ-_#Fh$e4R4EMYn9Y_BkZRZ{zRdx0Kb7m4UlW@cOP-0iY~C+gV`FF9>J%ch*Ro^Mmn>Uceq zsV8bge=MJ0du|luGdCu%cIO4RZZ+1Vm59^6NX1&gsO-bSzyN2=ZLPwV0HOcRb>@3(WfK(Dy{vKoj&bwdHTgS?i$jkM(8?};U5sZ zLEx4A?({D!NU+}8cGuAUWigJ~cK?1z`4I7cH9DJflyf%ci4~VKpDtrwUCR8L$2^;h zJYI#&>bB+aT|qzoylK|2;kCPnfr}h4mhaRxHsA;`7hPQ6(&4v~Y55!RXUTMV1J6VU zqK8}9&soh`De_ej4JDp^n`igx+aYsfN&H!IeYf(={?6;~R9)XDX1*rrnd(~2v&B4< z?_%<}!y5+29l4c!*pBaGGVVe)ZJfm272wVlFDKe+luv1V9IAv@WXyfr49?(8fnSXP zk0Zh7DDXNO{EmTNjkQiRrI26Pjhue+eCFbsDqsHP=CWS&_xAZtZa3-UFaFv;a4&6a zhliQ8+xc01cz*)_L8h>R6;@=kVIAST)Z@^$&Pu>>j%htH#8Z6_9y(ycae`QUAZ&iV>h1PGX6t%r8W0U#@eHYhLMjg%{qPwdJb#Y z@k&=>XbdOlti%h9`)%P3I*V32~ zj1;A9FTn>o@-A_)%(YO{!}u^e?ff3U>}O~rQFl9aUrOCe*^eYQLv+K>je{S!$ti}7 z=WV`C)Z0&;^@%#mjDhvc`3=O`cEfi?Q(54<@9A=1_zmRjO<%O@3t#9T5O1~ECzBqo zetS#&ox6xHh|GlN!|UN^yGx#zeLb7!E{(pN9)0ofZE4mKV7VVY9BTUY z?3x^MK(H(`09(w3-Y+YXlJf^5zd%DYdkx^f!tiBbE zN1t-Om#wJ4DEOn!nm)R4811Gp*F4qxE8`zZ&4nx*x$h{~ctt zKbYyQ?>#u;$#Fz*A z-r#(}DV$XX4*L%1I@OZ=S4RJ8(G#-^nfr`u6*6L?-CpFzDsXZwZTGqjtKp?I)~`&~ z{p_H76LJ&0v)7G0O#Ai3Dx|Z{3lG(Cnvz{6fvaT*=OK zzv08Db)b!R($Ra}X26aiMQ0ifEohB8LR%qh4}#G{?3u+H+nx>%Xlo-lKSJ3A+%?3o zG{5YOo!+%nqy>tB5uI?W`2BeL6J$Rm9T68wNy&UhIC!XHG-M_8gf4{`wzF*^=89Z>%jrXD{Z==Xs2)d}Y0jtH-zC zqx)%RHHt>M=~qER*X+WEZeoUvj;r~8Dlq0#$0l%Ga;9=Bqq|Im*WpX-H9xi-(C?Ut)9TK>p27cG}OSB(Cbne89AlX;ds zj=u1I8t-TDK9}-C@ffw~o|xDqJWjRoI1W7K4dOBMaJh}cV&M+EY~}*Dl?6U?9x|3o zj+x8zM45BB9#M?Ed5+uiQf4@1hUN`De)Gl1c@DiTpnL+hxy-!-(3Te()cFwh9LCpi zQ~|#1p17Y2V<%re>A0Fp{(`hXWrG=TXu-qz^j`Xtj&7C_JcdsTn-Xi^K?d(I}ifz+AY;0#LGSnE>jj^mBnXDsO@bzr?dJZyFDmvfEe09a}@E-hT z8ix5hYK9pd4dh)~HL{|kx7F2gAirUb_(iyqvrT#4+tfOHKlwp=$9B!>ZF*#OxRiId zA?Kv~dDqOlmf5|n71U!wkChz<9=oWccBH#wk;}#X#U1-+cFn0xPwn`sE2ZPB;5$<4 zw+7B;jbBn>^^^0dpZKi4BDWR2q0%~1QSSS}SbyhtpDMRDupcBJ;TH6VH97vy@8m!O z!;Nhl*rVuX|K#yBW2Sr@{KzRER+<6Xsda{FzM0dn_1n{A1iZ-WWo~c(k%u>ohaeYt z;Y};Ke+fI6m-WAz>w2zLHay-KFpy1kZmJ1S&Ns~d{p_)Lpm{I4yQjeWWP4Lyw7tnB z*IZsuwi?MR`cAYJt}uP=2TXK(V_;*`by4XY4MpVL&A7;_IuTo!M;i@IJ7+h-i&i!L zYIb9~tE0K;i_zu_jDbeh`;QCstewja&n*EP!UNhIZy7b6kbYk!thPKIFZhQ8vP!z`Ni!Bv`OP|8tZv z@ZMyj^W9v7I1VSacE_e!){dre(HXVn*7#u!^DivnylmE3_*GWPZR4NxE}t2d@5PIh zvG~v9oX}Tuyshjb9{lHI|A6?CnsWZY!sGJKE2W&yL@d@dJWSWS z;MG3HUGdnWnF{zivST#n!q&mM*~ZwKiVd%s9C-F`b{?~+*6huj?z19A&?a@Bi|jXi zL~d*3rxjKyxO=zBZ@o9!-+2%^lwYahjr+(Ca1dPn0K7iJGuEOt6~<)NBK8Vs`>f}T zwW4MFxxXk-$h~AW@l)|Dt)cC}6-Li#Ltanh2|E1Qle(5@2OH~F=yNdl47!({uS9Fi z@xM5_*Ilf8Mbsa2OaDq|{4=0~;n2ef=wc+iW0ZAbr<*u7cku2r%Y8d0;nxn_E!d~d z!v?=}R;AU$_;~O~YsP-;D!kOTPi4aIdf4wVuJ4|Mjd_mbaGhza`7&IzhfbCbu5M*c zADZH}VnxK!n;EU&HAd~`tiat@aCYT}xG!vsHMWYedCxwx_P6W3E^qJ;{S`8L9!VyVqKK(=0z8*A}2uaD5LW&w=qC0isiM?A0;<}>k`_G&@b(4 zb+ISJndw`#&m{ZU{n%}0zWuH?We!^Um{@9>3dMA5d<$-DhFa!EhmRgD3x2x zfxXfgxQ*xG$(LAtJgfRod}nzkTT9sOA4sFEx7_7cFSf(|%=H7K{GH?`v(MVu%R~Pf7MO_~7jwD#JC?|oLHj#qa29f%7v3NlDSdLe zPd?P^p|2;IhfCb+*DEd>J-Ri6IBRltw9Y`Mp3eJ;(CCuSpeNTme&@0!J>yE>odNE? zo|KpV4QRIc=96b|NANfQr*SebeX_s#49>WfulUkQ$Vk|A>sZU}`C;?ejQ*Ff=XxYZ z7EXLJc|Jto`Sq+r-aY>1Wp^X*qn{1H6B7DNStqpbM_<9-cz}8GCjE9@>hEmZzUmC@{5U^Z zNnhI+CC+o{ zYu%8(&ZQ3ZRdc^PsXuj-;x?anpC%^tX=9DkC-p~t(LUfj+FC+A>gPGch24WMX9m2d zwU*qj@T0FYXKrK8D7SDObB6I9%q?6tB)72UOb2kCl3O^Qt6sT<-~I~yBe!q{bB6J@ z=S-N~j*=a8=4(3bmfpx*rQQG6xq9WhC+BMGb*GuDtsblXt&1(??hy`e2ZxR4V^@pw z&)CWowz<-*7x%Ya?`5up+-`@T5%0g5JO}VZ@&jIG z^CfJ}1D-Lf*OO0eKs@S(N+W<>?O@kujRpQVpB*}8;qj1$`^v6@ z&q5DnwdOM7Sr(LCV=n8$K3VoUIuS3L`KOw9^00{!5^vr zYcu=<8#9($i(hi^@FqBrPg)v04x7_j;UfApxL{7O>$xBT9(Jgme`Bq{&!>#_p#~dL zm^Fhv&!q;}FX}z(Re|6Do?3ERCfa;4-lyy-V3!>S?<}a zk45FaXXrCAO#Lp$m)7pLUBcTluxLE&1Vk_!jUhw}q$tB3~)AG;fYEcFG&@N@%JNKY!x;m(&r9M*KnR zV(LotTmJOwcL<$J{Wjp^>USf3QoqC4d}o0tY~KglpgEmGQJ}K)+k_S=6RoDcXw~+Q3SVM$ir$rPq=-6;fe%~Hz~v=hvMyzBT{O~Ef@}`|?_h1aY);j_a~>T2 zWIa50XR)!M!yMlF4!T7m&u=+>F9VMLEASOM7rQlUWea|FtMRLAjmPe{V)IDOFIEIB z;vdrYbZyTwc!>>H8nPn(e)hA<__GXSAj-2c;+SfXt+ZELSjyPYu5~uLCGDltUPin< z5AAvI_Bm5&Mi|$%goLcO~tQFod z`2IY7iIsjms2a2IvFZ*c1b3bzH(gz>dHyl|_ zw(n;eA6b7{{VmaJ>z77F2dmAD{-@xj9`pju#rxpRqW>?^2OGAo(g6D=ong+TlC89u z81OK53gv^5AIc==80S9@)Q!aFf&EYE9yN=M!1zDYpRw?P?3~s<%0~D`eaWgftsZ2t z`PAReI%L>+oEzq-P05t%&(*vW?C)WR68yhAzufxlYn*%Z2min;uFFHE{)6 zbY2y9y*O~s#I^~{8bMTl0FD%1u^b)Y#7jIwt;|W~; zhIXBP<+Td$-gw_U@xFz2x&lq$)Y${b4W?o5N#Ij$3a)pie$fi=tndl$Lga#XrVit| zF`)g1I(#JVLpIEqTxlibd&%{~p}&M|PkvfkuD=(#SNg$a8)Np~M|5(ACEJ4dVS=~R zK||pinN#o=<#bMrt8z;z&aIyJZzESFp6fg9&*}L)JlEc?>@Y*iDQ}u`5+%wd`Y4)F z9TDN4HX@lB+lx=!zAyZS%U7&1xH{v$cYs59czdRQ;1|H4^2%)#<{v)!C^;R2CCrZm zOe*)zRHx29>Z^*^R~4`CZR%57sxzp1z_aRMT?@+QpIn#fk^Gd1YgJrQLWg-7x)2|e z98&iUx24!jc3yb3|)>QvKBF%iuiCI{L%@%5pDl`~w*0Y3$HH<p-X?7R>3G$;$w5NV3W?VjbiD#-?`AK3;CTFI(=!=^^ zrJ#GJ;umBFF0uE4Qqa9z!Ln)2{#w~AWFY2+d;z5k*WJ%}p6?S=c9&I z49RR86dqIWv4^)2-4p*e8rU*OUYY(I&;!}067 z6`SFIF}{CbkMP}I=8u`uKdzA4%b_;gxH_C_X(z7z2T*~{Eysu$>oOjmh^`+>2 zdN-GMMfip-VV?+p$C;&O+Mao)f8Ze_z27Z6V)-!pe#(i|Jxf^|SF*OY(w|4*`>ptm zmeda0BOBp7)4N+b`f9$bD4ct|5}a0H6Vck~C8qOqebIi83wxRTj(*PgVdDy@yv}*4 z;#@DiKe1v5_506~6!$6S^?$JXQw&+D60Y$7zj(^+T{=m~lmTarBO3jCp=VG5& z!JKfLv`am}d^I*IY{`$x#)$oOYn|)vXg+k)2#qKQ^j$ZUTlccouGAR_nj@32C74c` zZeUOu!ER#jlb>ZvNo?I#=FlPN_!zQg4>Cy|xRPyFxa~#`>HaXb^(AnqXAe?-KjVI4 z#YOb}PUQZJKto|h6lLoTy+k-30Rnqg|Es6oE<#$jIs8q&9VWz zuuBm$b~Af1)hY0XRN5K_4d5HF)5HhIz!v3k#=0Ph4)6m77t6#WSkG;r0S|p#NzUw* z-!LuhTb3KfmcoMB@BkzD+L^{$wW~2{pCtcL)+jf&HSpX6{PsM6uC0vdi=rdp9QxfG z0Z$&vh?lI5!8=&H#fOq^W%vtrUD2_^Ft{UZ87o$BTdEa;9K2u!frqOE$G5=!_pg5Ayf;JokDATv|O3h?yWGqgSlnz4?6n{+FwxIivArxzw98H57$kIKU18H-u+MbZvp=yw-NYaM{M0l zzVSk19)3=}+Q-ZdV)>%&Udr;PIKl`=<_(R4e%3d9c)06Yza@V7cDuGM;m%tbWKbbej ztm0AP0huXe!5P$)*SkTIj=Hr|IwkDnKlh&1%C!y zOKEEfzyImmzwvDi<*NC`noh743rVnhljrUl?rfkoi6;&$;c;xp<`ja3>Nry-iG zA6XtPHOrsRz)xEHM;hC7=v4g*vA>z{1Cej4a(uh!f5I0;XF?PZPf>$xSK`5s1bx%M z@54wnIt&l;pSNMo(D;tBXOPi%I_G(wSm9#t-v}@lwj{6JiS@CCvB`|Dk9Dr%XjMw| z#Y50%-AlI2FxF#Qv&7p&@YNdhBG%+b+rVjCEVfN$JdHbM!G}(}zAOcw3D|(gUZ>jr zGd};t%ezi(L$=X(pk*^Y4^LRats~QTkQuz)j6MGzb6B8^eZKwB(JtCAG7v$m>x^ z9PkF0Y3ISrh6a(d=R3MxR`4fx7;6o5NBO;rXV)P6kvrZxoMu{G&rbL(>K_K|;B#viHi9=^GJN7?LwQnyA04IM?f8|+zenrGzIJ1+_D{p`a8KbGwoDwJ zY?2?eVSbF<$+AP)wu7280-;JHaGdhOzi756E)$hA{*upZd=(jiv-deXqyt>t3BIwD z`xby#i+jh{&KV=k*Ic{5*}#Q#T2I*B=|7p8(E_D^+3tYhe;5=&5x|@m|@$qxEB@>RLkHtk7 zJ8~fQXKX&x?YVH-@BngCpd9&SC;JVDv7Kn%Y+&AWLtj1g!}xOdoDe?IlBHyKIP%vD zOTPVDW8_m3NPlNB-k%txeb;pvC5-cz{CL#oe6S%{xzO^Z4STRNr>yziHqL zvL+WerIVa`wsgWr3|%H`8te58zIFPOWw+HrTdK2X*b`2DC(>rB{yLZJ45zP4bnpgo z*^Tt8-G$8EZ7hr8%Qubr@H%``el(^%#wXut?#`m#tPf+`l%q)3IRi2DjFbKT8n7rf zLG*tV8clvz%{$qj^=_+;FO7-QhwFH!z8`>3>-|q+s+Y3^o^hW4jpwKHFIHL3V{&+T zd1}Du-Vtp>*1i+^Gd#xhENH2@l=HF%{O@mMoO6IVn`b)~UF@^xb&+YsRx-CMk@1W0 z+j`nHTb=eexU&{BJRC9~RiidiW&pgrN^;ZmZ5OP+oWWCKR(5 z5=vu@-TYb#s8N}nnu~=h<`Ma`Ibi}i?=ag&fqbyl*7X@vXy+AA;RyOG-(63z0=_2RX+Qtn@WJO# z@%+S!8>#CC>b#!1ucM8H=wH{Of29Tf>sD+b4HbWGCzc66*Un7jLCd{;rU#$bCE#}P zIQf2cBjJ;3e7s_DFp!LWE z#%vS5)J;aO9l`et)@O}Vb5nIR-^}YrrdUuXTBM)N zO<%V63s@HytbY0%R`BT=mhkXUZbiwr@+(V1*eKNJP2fecT4|Qe>j%87$tB2%-VwxI zc#WAq04`sVdC+9fU3uo$F7&g9;o2iTMsn3d$oA~1ZLfnit7uy~NZZ5r*kxY7)_!Nh zQ2?DG}458hkvb~GX~3>8hK9aPt&T|ktX&io7_?L>F~ng zR-#|E_;>u4`TfDl5jI}B2k~MC?*lKr;3a_z!KJ>3zdVAw*rQ*KJvE6EH4fDb03)^GsCei2Q_l?m0A@tn3Vg5%GbK&lf zs-yP+BV!o1U1bm2uHZLuJ5+Y?_h&lu&BV4ZUMYKp-iN49GSshmueqyvl>EGl=LXL| zZ60Y&#E$VwMMylz!}w(e{|)=6c%bl?497D=;Apq+vw~B>t;&bNrEC%QHMAstl#Ig= z`1#=`^(8BKHsAaSJ>`?^=!b#j3Bm72mZA@jaNWQhko=g*vqRXDO#ZK>9^~U~dN!VG z$;%5VuX*8p(73v+g?xpK%^~bb;$7;Wa8|@m`S(t2WWKznyc=4VGlLb>QGmRm`(JY% z;yc!@{(9oMe`fNG^97?GSroiLW%E2gs+cnKj^a-w%#!jRT8twHH`4Hj-)d ze+V6CA9QrYwuLzTSD!AR9gWe?p#zQ2&^->xtMDj0?&=8hAtg+{eT z9^L1+!iyz`YQG^X_>J3`S6cU>y}S`$^fcRcAe;`J(=QHb^9B1pGx!DJxYFE{tnyif@JAOj9|M0e-A|rf!{~@+>*d8ZF-=T@> z4^t0%@XI>7I6)KH!3Fg7jV04(wJ&yPf}Fg8(Kb!oVADh|be*6H`B{j@+ea8Pl%s2# z`^IQ9d8<~Fzp7=D-`C7OTru?TDQLTcJ+2uo>~rB0c~#5V_|cS7j&nWBOyiDLVlQL| z9xTguEvzw~!jG4oGR&AMJhh;6c~Xrrvgb#*R-G0(c3P;jp8FEuN^WCm3i%Nl*q<81 zdETtWUVr2K8K$SS8Qz$C_h-g_k6c@dzma{v+m%}?UUN3MK-WFL7CQem{MmyHZ1Yk4 zn7Z&S$byd@j^m&?)tI4pHxG5dCp&>{hU6>n5IFijNTY3Z)2@d3^j1FR*Gl+;bDl@E`j>V;U;EWJZC)LI?Mfd$ z@5{umlHoi`9iii~>4LL|`5eBvYPtAi1GEtN>u~%0IPC)zm<#Xs^Xv#PAI_{82m^!0 z!DP=>uqf^}`vcZhp7-)x&+g&bKE`W0Fb?f&7w5tB@J?%|N&lSq_ETfpl~YH0T3wvi zo&)cNX0~->6De%V3n*uUVkDoU{vRzd)^3yy`n@@l^S36F=r`xXy{=m}lh7Cbro(6vhL@){1Wm#)718j5m$j8eHy< z2I9Cw?vxIEbUQ-14Rd;d%Qd=T{#tO`=84W!A9dYJTkY_3-AArrP8=`-ZQS3DJbp^Q z{TBod@J?;2|8ClB1K zBy{Ob^*>t19`|L^bJC1$MylU(*0?Jve+B#{9-m&B61edeWHRWZO!S4##=xf~lt$dH z#`!ahaTDX)L_A;RINGy0F1Ywmz}q{ijhOz5ZGmDwu!-GFyH_^F_-a_GXzGbS0_i4o7X<(`Dj^2!f2&N;!yvE9g? zb^PYj%RVmbE}})jQ_cBEI(Jj+gomHTN7qi<*@JS|(S;SrQRUcLoOO`(M`tY{gW7Rt zd#gCB>ei=>z&{!9XU(rfx0uz@=g{+rcWF&0@7c3n;yYX!t#9LN*Flc3-y=7$-@0Hf z@A1=Yy@r3`!vGd?67H^JKNp!Nke*r|y$0N1XE;cHtp$zr4LJq;nY(wOZ#To+?;GoC zeS+t;U-4VcyPVy!qty-KHV54121AU?tK=?y0^HsSe_!d!vvcAqkC<@k#5>E+H8+^R z=?WXCojj&<2glTV%9w`nss6Mvt>V4LRR7iF;L@0q zi)(lNSZ`|yV>*Q~t>YarIQE!|URxN`66QocdyU2D)b}!`wKpWjGkdpeD<{WOxXKCM z0N4PsM`8sDI=mgb^l zN6AoK$Xb$}Sl1SOnx9eWZyo!NY`W5u1dH|*n8U~y7qVt%1xZY8%X*R_^}Z22V`soN z;P8~h{;}Hsw6;dkmg@Tjyzg|hodr+RI6Cvs;cY`{+c=fBu|NH5d~H2APw+L-_UY() z7510poNLj0^DmIS=cvnA^XmT31>`^dJz$6T4(ct1;Q3GR!X5B|r!M&3dOzi5k5~Yl z+SijT>*swtcAmwQUF$Mh?VKMU#V|oQwhY;Cj>0eMzT=m?ZtJy^H^tTwPc~3MnW6L* zx4*2Z=$c{sAgw`fSJ^sXJ6*pL>qaj44WI3|qTsj;`)%p@-&?O(#5eF){?i4k)*JX> ziY}j~o_6vwhnW*^xH9|U^$VH@VZ%2I*uvn%)BCwVGwrHOh}f0W!I@mn{|WFdO2Ydw zaDFX)K1BU@#&Le(5S(`p;yf3euN%bqG0G)y9s-ZT`A%R@#<}*)C4Zg{)*Qy+5c50= zeIWM@;&&GKRZiPzwVU-bg>@8v+!grVH6ah9w`fjOkxR<+j{i~g)WBTg-AdU9xD+0> znYnA=uRwlEJ0GX)pw}2ieu_Uu{{iuHo`>*Fo(Ih3 zte3i$&$9*pED{Z~t|GH6=)xwcbzZ)Y&Bz}f_J@S;)$oUTrqQ|@AHySjCm-kpZnaL@ zxGk7Y8?p(LS4DcX=&5iK(UyEHhWbzLmn>6BzgL=;-Jaj^NqBZKvN|@WF8ofWVhgt6Y@9YIA3AU% zBO)K#-*rv1b+O;jb^-cjU0i=OP8sWyK3xgliMz1l-3<=g!JWec93RMyJOkHTpJL96 zckcg;qa!5jc>jz0K5&XWKQOfHYeUNJ<6d@I(f^mBAFVI4nX0`8XdwM5r|xTcm-?^b zscQndt`YdR|4TeQ`2Pt{-y72RwV#5gZx1Qkl7y$s)8MIVql2g6y!$Ee|4Z~_e8$Gp zg#SxCP5u9br!NiZ`@B!VQ{|AdWl4D21-%TVr+eafdWm<-fFD_Aji=;dAA8PM$tFP@ zy_Q`fr-cQxr1(HuhxE;cuy7l@J@cR5r)X8mX9t_ z!yH5QIVhP2-e~(^)N(C5q{`|W?zc(S5%1jf%y;Iue7oZg z=P>#hX5Jy+&QAyac_pcVs~@yH*jbnL+`UcMEYyb~soF_wKz6F<3*Gh6WZ=@Ws?hPUMq9zQ1)^?FENezC8^}?!7f@ty}=J- z)6;>s8kzHVz)%eio&l%CAKJ2LI(z3C#H6e=i33e*CGRh}JUDkN8NSu5|f`Pj&>gK%)@Ug`T5H{OTJI|U^#2i%z&TwU(j4``t;nqS@Dr@1;j^e=oUH5y8*=m=#t&dy;Ra~W69n|{tZW-Y8Ve7%|MF(0}( zsy+Lo{f6%td->$z^~E@2Y9nj-W7s`7Kf*raxCI?vW7WE3$*h&&zm4|LchEW0`)gU3 z3m6M*U)$EQr*J2}M;nnf53}BCPH3E0FR7nZ&39URwAQaASB}0@oe|c6$B<1tlr5$2 zyP$XI?4agLEB;F>Wv}JB0NSZXjygcRvE-;j)L#f4WKxIvBpsui7{?u)ANKm$8^#MS zq4ECv2k?DT4wQVxJDk?MK1Lo8{0tVzho>05Z9HRtA!9d@F@4=_%}f6}0Ms zR-4%e(41rAdU`hR)ZV@1OBb#@W)6AJy{#*eadt5d_>dqoa89a$yg@8{ehK*Cxong* zlsTX>thwTOvyP|DkRG^$F?qi8OF37;9(aqw76T=D<4@z zd5ra~=!u*Hb9whS?q5J>NWf5-1Vb_{#%>oiSdzL=%Ed^ zjdM+f-?@J? zPT#HX+reouv-*dP2pNLCFp)&=_JEfS1KFa$+#vPe+@GO7bI{+Y}I)~ zaFoHkSNm;@(P-wsc==rF?E|h2Jim(kv|jiH>&U@KTFd%;=w#8DEAM9>V$Xt4n}#pB zO&7dG{O)q#`Y-0rT;|TR%$-W=8^M{EzoZ|je0MfK{DV5vQ;cm_Q~y-z zc2n0&8OBVlJ8GwrIyb<34gHTJ?*#LfHDT-1;96tkXRkcAo$=;>Gq@K&E8tq+>+IVN z@R1RW;|A7^%gL3YI+PFdvlMNGx+KC zlP_fhee<)=qj+MKdqHjfgf@+f?tdZWoAx+Qr7!(PMr*m5(cc7b+RB=gm10;+$Xis+ zeD8%G1z#U@{thz0bHJ&-C~vF!vXOt~6D~pb!|@@IPVW5(>@dC&|87{yhom;eBO~Nq5-bg(G0KY8_3aewYn!Owg0{Tp5SzoAW{hc= zZLqExV;X0RPKQ&Uce=|;$3O4}>U>G@NyHn5h(Q;<-G)uferEW1mLB|i;vV~y_N{JC z+!OOtK%0vb_gy~wzMA`3r5oQ#4>n>idUiT`b_Vl~y!dW+AdmUP*;}>`#mV_`8D%b| z>^#1ii(Q!<-<57##=;I8$Y9QBt@bkiO3CA%bCX|Z!!C1W&bvQC-24>wlr8)>OGi3# z2XZOvr^n`PKOjat#=2L+TIBHI%-{}qop^5_>p{U|f^fwDx|R z@3i(_2LDpqT6etg%yj6t8a~^XwwhdA>8&O3PvP?#==1OL+w~io(VLRGdWjOa??&V8} zeQ5q-;*pTG=I8M{x)YzsQEAp|T$A(E7vAjhdHfCYkMG1Mm3P+3qZYfcxwplwaB&8}%`n%&m)`s|HOf9Cq3+3oIy(ePu&KyA}E zXSX-q5Upz(7PZ_9X0p(*u4J}N!WySIm@^4*RU zbdgcOm;$a+fF}ibQhe%1_lNMj$~1gAPWsG6t1F;|a(umhZTob{4to4%7qszlTd2LC z$T=N~y|wAjKBq%xY~=)Bep0mRoM|GzcJWz)G`3C~A6w^R%;m>d$bSj<@^`F3H+h%2 z;P@#z@gdrSF<2)=OY&7bivQ$MXfDG3jpWJ}u8UdgCI5%fKdz>J3;HbMz84%pe|zmS zPh5GsCFe)r5ArQ2Mh6HL$$!&{HA(E#e0v>!_t+L9$W}+OKSY)k&&p$+Pp)GZ^=46r zeD`}?Id;7s>J4E(clM1qLxH*HaXb653IEE|of(&~`8u%Vv1gPyxM#FPuq-aOWb2dc z-8*F8_`kpqtKlI4?QJ&>5 zzSyVzesH%{K8}jt@Daap1l#YnNy7uptJs6jB;A@>6)03 zwctj+;A6h;*m(cKeI_v>$^8fq#I{L~+sC^CY$VbH60E_D8ngQD}HFWW!3(GJ3#!M&87L&1Z}`0h2lU`g+Nb z!i^J0nt*#ha326}e2w-b;3j`Hez{-SZn>+Y!KUi%r^2nc6vtMVI9tsG#&Tdxhu;0B zcXJ>8D3k6+KaeTrmB<&9b6ZMMux*;I4A~UtO0J+EDw{z+Hq#G}nUPVAU9fO;^LT87 z>(wX4s_0+866~U)m9jbH{N(#GL^2Bc$;smRAZX1Ybh@o0KUmSJux}hPc9{2@6y7a^|dDcL}wU*C9yL3Fz!8%ddgT+ z{A!D~8&mQ%FrR07dA|dl=JePCxGePaVpi1w3m?4t^mJOhZ3>`U_SI zG+TmiQ7P@{teZaKV=MG9&2)9lMWT75IZ9uxjBLG%2bQopaZctOrLVe6@!!T1efnvJ~Ck$y#5m8 zmQDB*zCo-9epxz`B9iaonmxe-Tu;!o%c-yS3xT{%ZmS>>PY4fq7=9goquken&#&TI z3#dOiZdCg8i4|9Y(<{O472xY7@eboRzOvL4c;&m=n zKiS)K)u87x|I)QbVsd7ryMZ$}ojABY>WjJE?0I_WZ#p!DUcJ+aKT|s%zVjkCMOZWI zp^XyO^Rj?>uwML^e#+hxt2TV~#ERWbE}gz=;`)g>tRZG9L~L*f`8o2}6zj;oRBI=5 zZE-_JU_Wbwa4sAap8<^3_+ew`4UxCiK8KX{n13^lvd4=K(b-75&9pC=&!kVR*-kta z`8=wqryU<>dwrwcO$UrUiFhh<$0$a}XUBd(f5?G$ELk1((I@8~*`ysg&|@0Vq%roy zic7)6JjQ)4>!4#V82WuSFbHSSY7ga3k2le*7u#DgI$ah0X-+d{D36GTYjTdxxSKdW z+8ywabK@TJ5A6bmdT>yWueNZUk1a`ZDhk>w^9J$*^RhRr?<<0)4P>X<&j*euKBNiV zA{N^+uN(iwDdYy!{4vV{OK3YxZuYPnTdJ;$$S)U5u?pk|x)a~%n~b$@fR`OE{6c6i z46msp*EBfk%ucoAGr@Isa2IQ);-12MBiUnjs~ZOyR1 zY0eKcCVlf8zVY)8dya23WfOh)=6Zk0!z3EdM|CJ_@fQSyJ<6(?-5)N(%=2e{h@U|0PYj=%$X^YT|R!^ljdhJdzH|{ zjIH2f?fnIT>BwhKaX*&!rg0XO3)$lc>l*t3gZg+a>)E156np0Qpk;iB?u$MUt~3Hu zH6EGBYRCrVlsil;g7)wwi;iavNtTBb@wwrr%WZo>J8PIHFeu)sHzNK~3jQ8OnXTH$`C>^^P8)qIQZ zM0?EL8KQ>^c<#i*Wd$cZE_!wLH;QBMdhoi|jQ%*6^2nR7RM>j0bkeL~mSlhGg7DS!>nKxbqQAG;}CB> zOkRb?LAK;dm_{$o0j-eiLUV-im<-4J&fG%HUgI&Bjzo| zk%gEGtp6YP!x1k2RbQM~wzIA$WTX$cm(TPk!idlEC&7q3@?Ifl4QU*I&K08>?nCz!~ncq0rPT$H5SAQMmt2gcCxWpmSOM(y4Rsrk54txopy5O<(8VAiUJy*YPwf~9FV{WN0 z^7C$ncO~jcz+Vmg<#G5`_kMl>eaqgf`knB_IJ%I&z>}nNFoy>G_+bn%b|>*?*nKL5 zj;4MR{-DET|NI0D2^?xXgh%;b-VO}nDS{!)ypfG#E92_$=geSkyq^hw)`U;3>}^AR z4sCkxH60p9HrBY=IXBQFY+9$yz)>e3P=en5jQexUv454etA^0oMci-X`;Ev|qQ`_k z+82hD`#krzL%+B2`$X6aU7rg3qSL^>>i+=taYJCr=KhY;z&>h7xeV^hf!i}9xi!cCw;Nwuh7wI zpTg!2PpUyCyxs*(K8!9?NsP>5^c{2=Tkj!{hiK$M&Wdb9cM^TqB0nBNzN>MS{}~(F z*3r;P8@9BtWWl)pT8j-p{!1$t`F*xcq{bDHou(GM*&O@_>VQvr?aiw9;n~W+HzPI~ z|6+U^O>AzgC65*xP9Dly;BQC9zM5~slO@j@TYHhqw07Ecx4G@QH^NuXrtajNIEk~@ z)c^3qDOMqQXOiodk4m`649uo(<;JV2Aomz!vzzNVYn}CI82?%5)!G{q5A@lvqaWFG zzv0okcy6d};M|eO4aHo(x69$jXKU?ppFWS)QRbrZu*k-I?1PgzvqCO(8T8sCH@Yl) zOxfGy3@v9&k!M+N7MSqjLg5Nt?84rh13dN^{0iPII=~M=I#>aEul|)oD#hzRZp%#M zlrojqUwuF>j4*b}J4L77Q@$z1ER@4z6?Z0C3`hI5;kYhg5tCH%{c{7W4P$ij+0ET& zYfT`BpuV9moH@bSj7H!`>z#a%iTCnjdp+s>4|$KhQn0V_l#eD)T-mHJdtp7Ir{0wK z+I?!Ct{&RzrEaw+y&fBso%5iTx*A=^SjT2^ay?Z}OU5AZ+B#t3f6g5H z^nIrEJ+hO6yDgr-ofRlE$OCLPvVX@Ofw^kE)_KLRg_=eYw>FaB2!6x)W$;Vq=jG?& zm&R`xzf^w2GA75mHDRZwPwh>aQQBE!)OK@r)7sg&*D+7s$dSZ_YjIp2l2F~U4oCOU5qXldUm|6tO znv?a*~%Y^7H`)wF;#be@AQ%G&Ln;f#NB zJVA_=lZC0!ohxYir3ZR~3!kvIT=%=#(YY8Dq_l@xAznk?QPs7+&~SX2TrHZ7FP+7*O5XD~Twuv+iOr< z`^kB(Id0>C@{)sJjq`9@ev8v`2wD~oni?O6Uod`GqyPMjfAv#zC*4zJO5BM9t_T^63r;o-u^Roqbli@Rg zFL|FvaGo45qr)CAd`1fJ=S5BoJPu6B`!b$p>F$pGF8xO6Iw{|G=9PZyE!s8E#WYU; zEf|=0nO7Q?4ZEQ|+>`ceTi>JXZH!Igy=`l!?S#EtHfz)4-2d|iM}|-K$yVReQyp7K zHFhi7TfjcX!2;H|UgU7r_`Mnj+2rf-u~uKbh0r=SELWS~a+h-+$;E!3gWEJ)&zZ*> zmkE7qE`ZyCkCm(O*yWI2Z9TOYUMYL1d_l`;{~T~G{Y2xUeu#e7$H(I=n|@A}&sA<3 z;}8bl4X(0Rk!`H>^P2a*d^q@i>DPC@zvTR5@5^>-b+qoS|MO4YZ%KXm{rU&re%~`Tbpg+x`Bo z1rfU*$t?AgjDUynC;&&bOX{M+SsUw|QFC^nkv2Z212lcQPx4`M_G@ol`|pyMSevc= z$O7U|oV{twpYjvfh))uGlv_L#$_Ln2-LmZ^4`+j~d^G=JtmH>Aa=l5?ySMCjvVULKK|!{e}d6~JdhL2 z9zlLQ_+*&tjo%3OD0ecjEf_}~_ZXg;8}f~VZtcy^=PIQ%?cwlH&1VJTJsI$8ID34xWXb$P?5V4aG(OJ^qa{_n$o2 zy1MZj6`VidT#(m!m*X03+dvNuH`YqFJqjGe(QWN{iSaRiI+$Cx3;U^we7p<0P#eB1 z_Iq;chTWHpt79)n_)xvd2^d<0{MBTvmF(IDzcdOa1~wz#-bwxHXOF!;ekQo)EMx6Q zjFss49)cen&u8s=jB#`La!#-;KwQ6M z!|wFBY?vC4&l+Y4o)C2?H^kxhsfYDN_o>vm%K2`Fa?JUtcfb2H!94yyVlDS(ZCD@K zWLm|@RUUJEAX2G(k=U~r+qOg}9v|DH-L9D)c^nDS`o&_el$00P zHc9(C*7eu*oz`0Fm7l42fX1PYbu9!upS9t^*1$MqaV;4_bY=6Dokrm2w!E4d9K+am zF^;d&C)vz?&UI|3bVLt2BKqM9_EnmsBa*)bTVOtWr{&OL7WP@$J`;A>KVp{;b52lm z-0ql5TUngZaThvXw&Dkp%I=Dn?IFKR=;4d(dh#pzz6d=Noy4Giot@sxdBw~tWZPdm zIRVR%4H<_z7yZUQsN7Aps~lgmnpOA(;g?^x*KOT>wqkha0sma~IIv~!bk_1Z=CQt& zJw$8pC;9tGH>IyOEZxlGtTX)s3$Wp27m!OGe)j4w3hfvkhmNv>kE=iUBH4Cl${&NC z2NwH%(VMmBniH(0UHQB`i*1;>GZRI4hWu!n@x!d7{5Ikbo?*rop(ECw%^l^CfG0sbga$d>+=cR~O*CB&VW9;pDIl|sQ-BMUECGZk; zXnq=FjG3y}#u@wlDi3eiV(%j}ceF1QnKI0ZUC0@|tSNnmjOF(JGJ8dq%GH+g-fcyi zOnaXwGR0*Xsg4c)KJs;-o6NWk7}0NEkv(+@{Ht&BFl!g{y@Xtv+FObw>S`=;_O9dg zCD-+DstbSc1gyQtKZ2F@AK!x5EAt8q2H~6woDtx(+t@gy4d9)kHqUclPHy8e+Th&W znbMhFj{yoNr!dtgV5ENOqU(^FN)PTqT^rGi)8NKepg#U)qdb z;#1z_yvYOP>$Y=SFqhze@x71iU{54%Mp!(cNw$=a=a!b-AsegY4)M$V?NCaO(aInUv)#_BqjB&b4ad5p8KcfW>Pe>l` zMrbfz)rBKY-+J_^8J!oM)t zh83;;B7OjIUMLzSj}1!RmRFzuTp;pS!+M>yG_Po6K;Oy_@)-NBf|;N6F?;`YBk=n1 z6CgLhjKk+CCm8j-KpDROwC^6{y9Um2PVOh;z9up5f?KvL<+jrL^Wkl&_$>@$|11q2 ze(GAJagh!gc1u<@gA3&69iKZU)&DQf`{?-luzZ!GRpg&6}*Bn$5^9?7u))?QL%NiY#Md;zn ze6I`nM)tod-YK@{Mat@(@{B9Tbs_&h;$Hh`AT{T*~kjBh~DP&}VNUYXN-4G>wjC)&S{~$!l!1 z`Vz+Li|og-XBJ;?ZCQgcs(H-OC67L8tWEYSk&JkdXHGfw-|;J{20zk4`gkY(4f}%+ zd*YUPh2B8L^7`n@z&x*fa-f`R@re5to~lQ(cKf{9#-4oQc1jyJtS{ky3ioBi-tXS`Cb-JjNJ)!CHYu36gSxIsgwoaXWYRAMR*U`;- zFtiTg&44eg!v8{N1Pbm(@L3CuH}KmFT^ZD!tS@^~IP;31V3(h#lOJvb<2DjH8O1z1 zHCLQ;ufuV>vi5erOPiXbVcA#BG&_g7e>Qt~^oOXL>F8i+W?{%9}12;Wfxx97dihV)+C0Al|ZUbkRtymI`xrX<@y>7$!M+!IG zpxC?;_ER4rW@ZCp)rH=74))<2JB>Yx8Pd5)?dWTBM@?w`{NJ%_U<=neQRViw8mTL- zoNK;dWnEroK@SJd%<~^te`~|N zS{(M;VFBYo=j=JpL+<7IH|1D4*rT#8WAB7|y?=_^-2P+?B-h)_nwh|HLNe3 z_Hb&!d+v%0qY?bWCq-85^XGb6m6Q2Kp3CP}I>hnVFl%Kl=S6|zX8NAVvzZ3x)B-oY zLStrJV_4HJH?69N-OHsfhKU9BQ+DM9WExfsY9IID>w#8;TF)id>${??rT@zQ0Z1-{A+% zjLBKZIFjE?{XbMxWqpW0)(2Pn2SQCR&z5{9JRN#3w)OBEms+7*$G&;b^ZtP+QjKj< zXizzG9)r&Joyj>((0HL}=wgS4a=C8fdHxn-ARoHPd=owoKdyP6oXe}+%OljynSX(T zLc`kO%I)tha`x;*V-vQ5b7VyL!knz@Oe-(Uy8Z&PLz8>CU|OWHI#6XDd3UZA0he}O zao`;a=ZEzF=yBy_f#-0>?tQB&m;dEkmCLKoGY(dAZZ$Xv9Njd`%Fl)Fm>;4M@jBgi zf#>`Q#NqtK1wvA-m`piPxerjKjm<&IJA5Oi`sFx>N0 zYG49%-90j|HQzmc=}XYt5!$Ld1bw5=+~m&N{W#ZM#9Pbv>OWc2m76t-XRj@p7QH7Y zuXR=bihUn4wlz5uh;_&b{(v&VTbJ@jPcFB1O`^U@Y1o*uS_`lzMX=SG#2Bi+0A&>u zEt=3bb<~HQa!&SDrj?)(;)%!9kDSmtFT)7T{yX~Yx7^EDy3_k79gZy^%Ugi>>fKB3 zuZw2<-MH%N=ZKwZbT1c9ONlS9eg{7Wc;OR_)leEAn~XPTJqoRlht?-T>nos<#!;EA z*}0jm3*DLhUhpEE?0=shw#){%r*%;r7kBaOiKWw``}+4zBwI z9B6J0rPUYuV%shR{~u<8H}JdKGeh5cYSo_&q$hTl6e8+DTm-!h)rHgwjfAAZMJJAv{?nPZB*Yd{CV3r}$f zYbwYex)HysHz*?;@gd4!$5> zz}6&r8N151&)Ds9CM50JHb0&dOSvtN=eFG!Ui1Lp$nPNgyN+MyROIR$ev%V4N9D7i zviBg_e*Oh;boO$~Uucg!j#c2#w2+}iNP{yX1Gwt9(w$yS0}*Pg$k z>v6VT0r?)K6DX(dTE*_JM9+j@Ay>-3Z10{A0 zY$|2$q^$CYbs?jwE#b{YncI2ZW&2&^w9Yl!Y+tjO;%;cePEVX}HC}DNa~HHBT1njB&ixT=d#9t9QX4-x!EM0vRq7$08{5OjMxf!+I)Wb?(6%?)dxy}yx=%`^LJ$OHeLjnjs`iFcCC zU!$Jz@9}jXZft#>Ywte!?2`lao0ZFNL6&-kevKHBuDq6kj&0=2agFGI2f4fhx%`nr zV^0S%`Hr|u{toucOOd@xIOo|jBBRxVEYAM(9_2TTZ6FpI`QJlbo?iR;!7gNsblS0> zIc4pfbY7n6yD)yb5&VyYyI$t>{tZT8A!{Y^W5{IKnAu+rKmA3kAN%s#m-q*6<+}~$ zi2j|_drM>G@}tP}@Mf6tya%l8cwX;AM~&g|-B#H8V6`Dn6RWwj@wnFs&8n zWABd>c-l@96vH)#i}QPb&dk6Nwf+A7nAe$e_HFI8*Is+Awbx#o%NDG=<`w(3&R4U> z*G@7CS!AS@e@2^MqRk`wt`-e{6qWDyxRLKeo1(TQ*1jM(tmj@T7x>%7lQaGn`YygS z!+gj9d&-gPidYLw_^j?VS(jv=d+&dEoabZ<@J9GyM2~))wD&)jPJ*o-d$e?i7!7o? z{!4dgXDrXGI6vM1v}x~s11GWTVlwyndn5K-lgtfHOCC4YNp@#V_{d<7wsEzw?g!Lc zeK0)zWB%{rr#0vsjNe=Q_e9SPY7Np}R*TBDB>U2-NBTv*{Nq3RqVGArml(r+*+c5* zdq~TQt{E}D&Ut^k-<|YZ{S4-ChV}6XU;YboEFKpXQCuj`p=h4HC z0r`#FeSPW+KH^k-wAhm*AL6GTt{!FkIMkra2I#lq9{t-g9p}pYfHGQ>|81G9)U)k8 z6~!-n-+B(aT-2ZWsfV-D&huyfYID?H(`5LjKM`K9vG@XIZ5;9KGQJ5v-S9%mJ$a<% z#-&eII&0Zo;CUk7V?HS(g**I5V|V|38#)>Fq>(n1pJG>?{FWQtHW3>|68k#D&RCIx z@1m1^sfevswv}RdjQrqA@M)L*E~}h={as)`fj`|bbiDs2uK|5Zzkiwk9^yCHxZu3Hf&aRE zXXU&05@mu5jP?%hnsXV=^EZw1^sj5gADJ}Sy(CA$`=9hFH-Mov=%!af;7Ej4& zTI0JDWS974ye7u-oL>!f7>tSLwf4Kuqq~;T_TKA(CiZiw_w~n^mxJ@vi#APs?IU?! zjL-A3?T`P|!yf8ca4ygD@p&RX`%gU_q@Km+@_e5>!rkD$=MU=p74|!_FmxQlwGpNJ zI{IaQLq?R0^v%`6TUy{X@_vJ~PsRy0@KfQgaQb>Ha-PcCs9STlIo7*t$SOs~+O7*agVVe*1awfLujM1NW`RWZt_|{B(ggXzNia+KG z{1QCaaUit6n^0<>ZI`@x5wxmvING!+kBvq5piLab+brf02Ty1Qb2{V|{ZNI%kjg8Ng~j*fKS{>A9KtR3SSFX>a+ zz~zdQt3R`B^|`Xil-0bnc^UP!g9q{Y*T6yS+qNif4C;T0@A3{0FZWy6C9Sc_bof6x4`xoj7E)gJiP zDrA&A|0jKyvzWiyZ?7kho>|dnb@V$n1N(bN^t}ZhYUiUJyIl0U5iJ*tK3k&SgVASJ zv|KP+PHTnAcSP&=M4$Ds@6r55^jRN$cEr+SaK`dQ+o_IzuaDImP4~p$iN0q+Z))F& z=Btj@Yeb(}vGUP(=#g`X9t%F17W;oA|D{WxzGDbB>`%Y$KZAXTQ|ZK2gdgj>d~!mZ_k9L^Rytf-y64IP z^n^_xJC_|p$F{hiDT}$>JZ{JW?epA9tVma?Vej+QkBsd7bdEyi-eUOdd6_-d_aL+< zTlS4<#>|yTM*qPS>{*VmdEJs zIO{a`$ut9F<{9U*YUV+6ddHRQPt4SQri3O?vWfVfImFqjFf;7`6NwYnfR5LL zZD9uS5p$DtM_MH2f)^NTv3E-DNs6m?RW$a3>Lo5L@o*G(q3nL(uwyfXnwmEde@S)zl5|(5;Tzlk;Mz^YjXOQSRZTmC?IgP2`Scx}KWK{V;kzF542WK* zA1aywPqXetmlEy8`kHYtVMa4{JXZ~t$mJ{Ekp6rX^`1@dOPDdhw=copw634N!^yn+ z^gDh>Gk>uayu0ha*q7jW7j{+oS5m(nn^-V|_nGps^gG;M`B*L)Vt+xYN1 ziTGGLjkdo~r{wD^!Epw*D(Qf-$!p(eiS7G1zRzG!_7rupr{7;qTPv?IUdiP-o;GWp zYSZPl^V{}{@qO~?{37+Nz{j!*f6Ogfld;v7VLRY_V1Df(`C}UXTIbl+#9p=iF;|=( zF+;YsY}@a0e4r-Ub}@81-u9_X4g8SvO&ZUW)b(fTDuWJOX8T#6N>6&kr}A#rL-|zN z>9$X0iS1LV{;R)=0T-J1=kt6x_5CJn)xRdK9)G2Z<=pqFe6;04S7C0fb(u>BSpRIg z)EOb=`}{U+024j=)fa0H%Wv$Haeq$VL2R7T2;HWqKWv*rYT#w&UzPB<^ZauG@^Mi;|3kW- z@A3qu>1{k&Gi^Jmo4Xx)&@o-y+X@fs-weMRY;UmTwH(7&hCE-3UySUlQ?RchCvzv% z1kT!=lapiy+s)O!s2iP86G)tg0|w!8_R^_1NBSu*2M{8+RXqxBhF?48NHa%UO7nGd$w zekbaa{7#CIO|;Ih!uMP}Nq%Nm?%X~w78zxeBNBVhUH{15yyJnxJx0f2#U1ZmziC{C zxq4iY(^&SUkxy?hfNMGBUBy5zC|pyAAu1wyfMk zOtm7;r1VYRG!A*{&Jc0dGR?HQ5bye4J+j%3H>|PvG+8Akrv&j+d-vgpZ2ltanayLm zW3d#X=e&t$ApaAa$7GGCKZfrotTC(;?;61?qPE~s_Wt3|WuJ*ZH^g}2lVd$?+pVxo zfrEPX5aQ`j{PSbzO3#m?Ptz5#Va4)w4rPCVwd|AYnC_>G_=2K$zHD%Y7W%YhK0|F2 zzj-(KHay0jTyWtsatCVhGt|0OgB{NKtVZJKqwfLo# zu}9L!`jh+M)?F*nlY5CpthL5PA9YSl{9g73`Myp0U-ksqs|o3ySQe{>dDa%8YxJgm zcbuPf5*|5o_3?++_JUi*E$fAUMAo>pzN56!0KW(|6=%4J6?r%7VumF}RS;+CCQpjIr+x_~lF1 zk}Y-LE6fw%jjgfAuSXu=Y1%eT$&QV(oo5a8{fM>lVeXL8I31vz_`2A40oqrm^zy5LfX@$Jp!sTl33pcz&9VZ;8g(=5J2ngqSID_le{&ucsiFOJ2@B z`T%k1Ik#}S51Ntuk`%yiec{xD53H?qd#n(0(LugFj-K$7p~Q{i+d*eapqX>*wQj?@ z#Bi_P1Wr5XYo|GOX7VGR{57nT#FMvXqto0F)WUaA>*-#F%-Os)Hek>$zyCP%)Egq&Vkoc`KCMQ+_X_E`fXTZ z(@70z4u>sob9S}r(WorT+M_mOb&FR=)-sQ^ z82PYFX<0_T_yps2p4=}u)ZU#hhtpLio)2n`j+O2Cvn}g=rrsbv_+=a3bLS~JCy-W) z{r|Ic4VAf^GCP^4;%z^rKdMEG>W}nfyI;iI7reryY%jYWH3sZ_ zE$8LS7oL#QZ5hi@9K5F-_Wt)N=F+{`#%tN12rDKacwy~$H=B5N@}YPXURBFJhihcI zcl12NACk}VaD1)QKjvJmL;lO>&ilsB^$Vx1)Z@zj+*uje#XLJ3hve6hSetkJUA$0s zC&O2d?E6$Bu_)r|eiQg~J~}>*3jM=p=Dhw{A=UxmLA>j3U{ZW7)`o4sxJU33r_{GB$!Z)3mH#-G}0Wq%5~A_mmwd^;bBjE8U$f?l-0V}9_!KJCvW4~fzNeRlJ{Zh4hq zHRwFSm5L*t8u(&5=kdlcS2>Rtnmp2Pr`;M!8%5fQvuPJZ(w@1>cZ_fSXTOh#d`~X0 z>l;d$-m_`LBWYJg($vm@SlSE8k+g3_(hlSeB56YkeShZLduP8VM!pvm+HHmT z{=2hj?nv5SBWcI^zUyq7D_Y-|?Q$io1xra&KSHI3HHN<1{cet=_wXIO_aAjOY#1F& zBL=YW6^y2xAZ=_cZC51i&1jlM+PGNStr2+s7)?7x+JsnIOQG*bk>*Bf;3uSq(@iU! zYFg@(&PVo5o#*eHn#%7werf#X@*B->4!<$}zOA!)rt_@iIhJPy&kUaBJjd}Y<2l~n zi_horsT2Ii7F@{Q@aM2yW%^G$P3!b}k98`+w0e?ED>Q<(Gx<5)@Y9d>uEN&X^Ca>E zXMa7TjfLlEWB6R&34#0g zzLsC;F7$Ky9jc4bsPH-=@NLq*8Y??Fo^B=tzRveKXVcEn&1-xsJNrGJZYBf-d-2({ zc)FPosO9^Y&ZfoF&4hsH=C7k8v~iqqOpT4BO*az)H&gD@>1IM;el+cqbTc7vX*BJV zbTc7P5ls`_guq2S?MUYN9O>c592Po+9Ya-dU_cGtaWl=*M;o13=aOF*05i&M}IGXmASwRd)1zHH{kt(-wm zDzQ$bmRO-zi>;p5i-`s5>dIh`&Z6GXmSQW@8Cj2m&J^sC;{sb)Cyu^QOibBS@Aw-H z|5$9C1Hg2Q@^_UOzSoFRmdRepj;+SR(~mJ%1pAxRwH;V@nwAAD1I{Gh@ttFRry8BU za2IL6BW<5)O~VIyA-D)#%sJw}Fz4Ur`vIO)=K4d8v+_d+jP|~XCH|h5hWmTBjKKa# zzjhAyRsCFeEwN4}lvtea_gXv-m0Wb1J`E&BU&$tmn6L z@6`4)_|&_L?{8jgEc_jNFNd;!yWzAsAuwmyl{4zdgJ0dqH9RHTOV>+xqN7FOJ4zcj z0mGf~@SQjZUlreJcYc`s5xkA{DSt5GYi-z*+Cp~~bM}aRsRhZ)LtUQs(Dlj7Pac5o zPdTjGM8;(C*!)mev88hYqVZtqa33<-%h??l`F;+qhJIIUxv#FD-@;il;lsiIUm(l0 zBCCZM+wUTy_c)Wg`e?rgoctAE;rnR6Au0oR>=^3HV6KEJOD*n3=+Ej0_xF}Y{8v_# zS_$e`Q>oP|yTD_m$bBxai7(>uG`>F>&G&Svbw=+SORd~P;cdZ9rIwp`$mWCYjR*Oz z`-fJsf0u~-qCM-0lh8TQ5fX>j&(F$E+<4?>=(Ux5)bL$A{g~}fvruz0#JB~C4}7W$ z{pe!FfH2y_UHrbqZ!W)mMtkpytJ@ErD7E?ymBKTSOU>l25c@fyd+q;Tbb70-uxy`V z+mu~6c2wvxqrIxQc$xIWo&%nC*07}qUvm4KiN6(^O1UPLOX><;&;J$ne{X1}Vf9w3 zyl1Usg6Z7hdkT5eFgSM-#_zGN*m9J6+U@aWjvd%J!Z-7^@cJI;H58p2+)vZr@`SOl zcLd)KF&2H$_7Ujxgb94y)$vtxSr&GV%<>2h4DF{~OPpP78SD|+@i|@)j;Ygye`x0A zM!$&tqBD{;TorIv+O{ z?jU~-&%g7m{obH9i$A-~EMKI}YTCRb+GbB`+49V1js9uo)8j%r@iiFDeSwVqe~{;! zlb9)&xwwlS}=s2NO5uA+MC`J%>Fk_C|t1KdBP zo{4;CKJ@4DoiSPN(z{@M%vg8@>GGKtzs=*F^mgfZJ%VWjbPG)K&9lotI;i|3+-nkF zp7*om1=lr{2QJ}N_{1LPTUP?#3&9um5yv*km~n)5j#jdd{?G;5FIlGhvhp5$U|%9@ zP7=NeOB+^i=&K?&$&#f54=h;tF6 z`T$z~AOTq+i61mAx#VCf@&Gc)(Z#Oip>MgC_wBoY*d`-v-hag|_*P%Bb@Wg%bBQ&R zwWZ_p&?fP=I}ldnnjU2;ps^2gzOS)Jw{_UhNRp?gKk+#Iw&|T+`;5VF~#`eRx zoIz$iIP@(~duVI1m5ZFw!}_l=t$}yje&OWPvkzHs_Rvc2ooJVIEQsAg72rY!S_GfkX^_- zmNSJPs%!#fDW3*UDl*ghN`U7@%6Dd;U+#~Ti`D6-tY}B;=z?L~-x6J~lrGst?Y3}^ zjyvak-N-}xkq>q@n!bJauq}JHS9GPdu5sB=CaK@i4($EWfw7@c1^%O#Q!5Z zi>u;=I)ff3*20ojPIKvj4$(cZ%QG=GaZ`F4M zT3XjgYyf^Mn%GlBrjhJ*ajvztx1l)optcJI($@|4zJb{tts~U!`yi5SHKmOHG@mp^M_CMY`HGK_xiHs zx!JBMDe$!yGgj_vg+?>8xvv*^;BDh}PsC$$(ziGM5pIftAjR%k>@rOSQc zDA_Q+FlpC@)6k7wR`jhk0lyw>A-4ZVH@tM?1<)${|0|grIh;dYnQHVeUW5N-ViD&~8A4Rv)_-M1o@eS@!f?mI-gXX|pGt7|58{RQ6`-ILhMPxX~T+beU7 z{+zE<-;oE`f{T7n6?+Hp|4R6OZLw=gB|LdHYeF4sK<#E%U>$3~ic(|GTxg`MkbL=` z{Mi|v<=*0?DHU!{dj&d} zcw(KeWIj~h#Mx$^k~_WpKe}Yel`E2rJudF3%mWX0UEt{ud4wm~nh(vz-^$_Iy8l0Q z$EwTQgH5f-yr8d^_tT7{=;GB-AIpV`wNm+f0wX)X)CA6|$< zIsPpkyWSX&i1Bps7B}|Fc)qTBH19q~U6P?rGbbJ&Wvr7-adW0I>Z{?heK&=#+gHl_ z0BgfZaC0N)6|1CEU2QDn9w%D;nlC-6O<76hw}7uH?b zg4@aS24@pmKZiV6JdXXR@yLhnz`)b+-}T7z8%^uTCVq_`>uK@<`(E`U41XSgKZlX` zYtgY=C(rP2ccu>2ePcL3Dw~^l_~s_Zp5}JO7CMz~eJZM3OBZNq>@3*a zsJjElw6!$uEC{|?U+8a~HPExdwB?^_d}~AK)!h3+S?npuQqnbBi1D?V_+G&`I|^GG zn+ifJ=v(8E0qM~-S}$q$mH(#A@yKS9qu5i5>8mf&Z|S6xwcs;-wmfCaU>SjY*1!Se zulhw}BKjx&_#Ae84siQ8pKd8eMm)YwSIVy94uq}&AEn@oKFilQhAX}2@ZL$Ar^tIU znveLszL%jn&B0JK#`Dpq;cYo?Unla1>^_nYf{T-Vmj}Y@?KlX)%U-)3m%y#~5zwac zugiv3cz@QWm9YW!Pjj*Z`M{=;mWzEq=G$gu4&@If+{nFeaqohqwnL@R zGJO0PGC{{M?xG}?S|@n!aQv#EL$TB*o7ijMMc7QGpKWIR2FQCOG||Cbct@_{eEg%s z+d8-_r}`n}j3R6S4wHK@i0ObFvmgDxs{vcV`=qaZyufwDGf;aEHsOSYZPkwwyEx&i zJc(nBjkvprI0_fk&)-h|1nTBVM$&G=B(wdhf*1{?H3^{r6uy z#(($3biZ`m?@Sy^jG{T@%OL*jY@Xx%J@CuWV(xgGINmRrr1qf+s2&j+j5~aNkFL~M zu#Zk3n7`I4`YC&GP=2uNZ`U-1*WXQk;jc!qnBKk0*MC981{$+v+TUk?+1JN+oqd*% zv3zD56wB(mNL)PK74<0Tf8`fvQ+_#no$31<-;x-o&;r96AF;Q-oc*rp`vc!)Ydwu! zG%fc1uMXckd=K*rH5n0GGV7+@R+e!=%cN?Q?4 zLszoXW;uMLfa6utY#1loHtd(P*=I}Udt2-~`x-UEg@OMU?Kx;~E@^*Oor~F%we7Ol z*0A@v^hkP4Cyi;^_#&|*?)W{kD-Ti}S*qKdS+0VWkG_=)dKXMo6&PpYFjOR|F zvA!PQ^&cKKpuPl8aNlYa=3qh^{3-%-Fky6?|M1lV{=>#V@C5Ss`}Kvvu8zXpch2D( z{Ob_q5>~W5H{yXd=dy%QV5}f`sH4z-7`x+P$3XC(l>28#q2{0eF!$+l z8&UAd2JWarUjP=(wU1a2o}rH?iQRF6n1|AF$H2d|E`5pb;^k*J@2_?FG;&J@{~zG{ zJ&Z?Cy4m%_b3tZ?N9}bRsoo@X=Y!C+d{4Y?k0rl>m7Ixnux_nMkbVXSmWtc1E5^A)R-h*1!*_YwP8Pzmj+M6{$NB*(s^7{<`qOe*&l0 zH4{7e0}o-3B>&C8`4RGt4KMTj3w*Qx&-7TA^1ljrC#FQ;O=>cb<;Diw>En>~k^)=N ze}96$u@W5+p1ANQ*zQ_Vz(4y29|CuZbGWyK_|h%Jwi*uHFExC9;GG7+J;H{2IO)9G zaDPa>zkx?=9jg5-;;7TtO43~XKA;bZHB^I5w-s1V-|Mlo?w`(y#DzIcTateSw`8GT zkVp8HkBOd=Cna0z{~^X?Df#4EY`-HT>s@(Vd~+km=)Z7To=y9dQM$@u4+@+{R#zM2 z(Thf(DciS&wywgSaSS>*&Hd;>a9m4%_7d_{hh#{t(WgehGpKhx`D5~$;C-6+4If`e zoQ%}KQ~YneaO{F9)c1Gl(R$krUS4M3n_xicD&IVe`9_+(24zk14TZL{@DY$MD;gAE zSV;b_Q&%PJ%NJaI7c6?p2S9kyGqjks1lh}AkNI8AQ|j2p9R6}NU3}n)6^f~5r1_v1 zmC?5yd@JDFn&`Lp>03VU&AjUzl${S7^as?n&aFGy1Xl(=6Zra{tw-O+2X3RR*5rw# zX}+th(O~*+dMLbJ{-JvJM4wsFXKc?#^(lQ}^r^nuZNrbg1dQTIlA-l2)~BLa9{RBi zSwVffDGHO#%Yb`3a8Ki&wF^mG5Ka3ra7w;-hI$3pwbAcs(fTC!A{#86zEd%$GXfs+ zS*%wl8LLl|eJ`WF!LsiZeUt1fc{nOhPU0Ihcvhb2L|>8|BN@hM4X;n6{VBkb7llXT zbCu+7`o5|`V>-n5M_>_-r{SX(LMG7}hM2rj56s`74dKeBSDuZ$pXK4a->Z7y;os2{ zpO3Gv#5bSzlKK5h-+`v&J@UE8Yxz&V&MC+S<-!jr-aeHK{4x8~t!Ik&J&tbk`Ozuf zci=1UW5cLuczi=0_IlZ7--cU{XOnhp0vFBU^ zPpTz7BqtHymOYj*WBaexrsW54!TqtR2|hOUIS5U>=G zKbH300rpn-rdUEcqkRAx%wk{hICAE(smP?4vClV+eXp6wsaKiSsUqZOWY*)z&;!WQ zrxz#Y>)gmO=8$}Dx~XeF|Es%@2M(0v%Z}4>Om-@#5B+Sd@E1%N*0!Jf!|ETMMSRr6 zwt7cbL9nWx_Y&S+Z4Hg3#7TAYBz|gXec}Fugf_{IF7v+g&RB4dldtjx=1tvP|EVU% zXK`76=Cx)dmKFB6Y>j7XRFCW?CPi^1&Zpw|f{XcOPH@Jz5c3aTrnKtCF5ku9X0Ji@ zP(BM>FBt*PqmHHI@68#O-;FcZDDh#eV8s9=r|CKEpbu|9k-k;pn zST@kg{$vL_RcGTbN&8Zpjcf4poYT1{9UKLRfrEr0ZNXt*;W?BiI7o2t)Y)6k@J8>q z4h{ns+%p}+X(u?no-u92pRpu3k@0f{0uLJN>~~L&{XT|w&8Juy(Z9y%3R}-k4cy0k zRJ@bp*C*n?$Q_+4IKK~0RVJKZwC|e1Tu3#RW3zc9oM|l21qbYB<+ESqBX)`JYky#j z@ktKhgCFAV-=ib2-w*THcfXq8{=3{c@;!W?b|J^tibp|X@-yokbas9B$&cX^X=m?# zeT#SbR83+`j;4FsBj=gHQS?kRXO+Q4eWA|OtK1{>Lw@%j+B}BtU_j@hXPuuf<@bN& zQJkG;Bl%vbh9_3P7m4wwaSzhQe~_j&UA$iv{U4J7YWc2q#M|Rza>U|s2w#Kyv~EzJ zp0eLl1LLWqC#U_k(|4Ho#eDj`_TdE29{jVcv(Kh<`~85IJ7}VADm^5fHp7n5eftyG zqdPDQILCHn!oQn1)2Z)@#~J!3-*YlHrnA@5J9(Br@ix<+Hp$^1mtk1)V|b6bp*ow9 zjTCci+`^JP`PqeuHLcjyH-(MqtZ}xF1@Q~oU53lMowEcJIFl&6h%Suhvo2HYc4PUg z1-AX2a^H7mcsUC;@1_yP^z9R|L1XWIHi13g9AlZ2|9(eC-?3?{Hf+IGx5~-)TR6vv z4?+=k?QHl-(E(%GPVC$D+>v76OZ#*Dru)2mR|Q%RyS;B8mQLRQ9UISN%`qNM9%1L{ zHaV}}@#nme{8F%Yo9wA)+;FlOI&RKk{&D8Md9tUS@0`124|a8;CJE^`8>;Bp(^{9zt&^#a#w;7SB8?Xkzh zW#<8|DZo_$Tou4o1YDjdT%2>Yda!-08ZRB5bN0a13tUso#3{W|xa#rg2olE-c!AB` z*D3xKgUyreI|XkS{)5C~+ysp3L#y}*-(r50|80G|Ui1>0JIf!V)zft){(-((R%q@J z|B*~j`=`;;jt^vunrzFv;?7_#xBPbpd`lm-91-DfW{2InpypH%K?EHtukIhy9IWXh(We5FAyEO!ZEs zd>v({4mAhtu|d`uXO8oRGQrV^f!CXvcgw*|koRon@g4kEn(UUsQx@MeuYblENw1Sk z5W}DJ@JP&Km)&=jJGJ0y{|T)z!pU7b3WMX3iG*Y3M2ImnYIMeNvd@v*Fn=2O67NZa z_elSjeJ{p;H1|`l_xQgGzqp4rR5H+e`|&lB9Tquh&c7#b^FQw2XQrfe+4c{{@(R*krTx1Wp$`F@{e5b~{1o6?`Qi}k zIP1R2eqnZ&u}v{&P9=EQPrK0Pk~{!T@{pk*$U>F|B+WFGhze>{PWvWhvZxeooO;kWg!=gq6F%rtEN4;l-zGK_7= zcJE%1QG)+Xx-FBsfFYN>`kq&FfiJNF{&Nw&1>lrDiFYfwuR>$&VZY9PFD|vJI2V-- zzH&8=l+~JNul?K=D_LOm{l>bP2}W8Ze@>)4GOqo;WR_iC@<18o#p~?v4VT#YsBK|6 z^i@O~oAC#!B@eu%@3}G9#(`}cFs<8hfiHEQAG^5b5OycNzxDvKQ;C~3CatdpyQJo_ zV(2LCMbbncT1WLB;N31u9=-pVci|%l53=ibjPu2p|KXtW8wQpC9`7r)#|M4cb^Vn<>3O6HQ}zb2kjWpDe66}p-AlR4IXuhN`t zN1)HUcFgppv6h7nc>H6qn;=K~jEe-iM5sSzdITtXN zvj|1+>8!c*l(56>oA@@g##{{HgO<97d!p>IOvQ$Y9Cn8OF6Mr3-Zn8j;2S> z6hD=KjKx~98{Y8t_8-iz2)EyvYifN=th3`nz|$okj%UomNgiLgDIHxda`%?b4GmP4 zBAbs3HU(Y@XyzgaKUPSKF zdTwmR9;)+co{`Ll(ShHjATI@wi!Sh3%=wq|ibwhK@;v^ea^~`+q>aGYH)GnS4JCeK z;gyV8PXc#P5r4G!S;-gVFDzjX!rfQW${d@@JrVkrUt&!|r>seg;3_eSqfF$EhNn3j zN*ukV+~XH|%o({$=hZ|j^m?Kd-p=n${_o`Z>qKimcB|g;hSf9Ku$Fp_ZAICvW%zwb zCJG%%!EaZ7PUun(Kel((@V3Kwzy0mu-Pd&6a!dV<#K+-#-s~3-FG)W4-QfGGYu4<| z``{;s^M3Q2v-z{}T?=SREn6w`btU0`6BKy@&g*LUoROmxr;OW-bqjA2Y9h@fYTk z^IQ3@3!!g(i5Ja`@a{kT2_Na1;GH}_4U%U|asE-}mF9qC!RDy0*L{Vt@ELTyXVL3& zq1RgAlMPS0UN-zk>-oeacoyI0oH$?CtKfYTdt%c{|M+z83$y&?4emf*=h#3`4zYe; zO|oip&^HCA_+9;lzH5IE9Pm@owRtIVLC^z^B8Omn-WDQ{tMfwJCtGDq)=x(xK2R%+ zb=I3n*3s=r)r|Ox%|h55vWEMx`nZ{_l2W%ncs=%nEQeK3~Q=z zp*2i))(1;?hcD3InTnyk2)=M1vHxXXXeO?T`<1FUT=)XN{m7d6OQ&qTlQq1~&m9ij z?{{Q*B@cLiC8;_t{Wj)3`296%m^62kG zVAP!|ul*HU$YWRgZ9LR`9yl`si`W|~sdxIEtQ~EncQbw(52dNy-s6nNzH6=SKU{0e zaaTsiS^QJu+{rlSa<&tE_nYW@y|QbcsT$}d=If2vr_PQy_OVZlH{bTEe&(M^`{I%B z@cs|25l4nIw&U_`e5#N2Gkx#vi|}=gZ8kE9^}DP6`J-dwIx{k^^gA}L&oUm?so~a< z@NgR^2htx{9)h2{xQ`-&o0&Flj?mZJ)%U6Mto@hGvs!jK_gL_h&Qsy_tPUPUX`{yF%fv6MCv&6~&KYS1`DXKX?=1gE zLy>i+&bROG&6V!U8dT021kbLs;2&Cl_g^;OIxua%m3vW1-fqtLF0D8EcS1kn8BIJZ z>kXe`YAC;5H+K^r!7i(Nxo)H1@wC#7Ey%X<+O#s$7knLvj`IH>=11b7`7w-l-BGErKLU;i)8fa+ zBIz3c9{QSVM)<{MaDEmxd4T_GPn?*mHqxdty-~a16^0O#xF~0If z6fd7>GafIrnGx4#{6TEFCOnxrJ(YPpjo)SbpULwo=CnPZE<|RF$dxCDFt6!a2F13lyyj^)Qtk31ec^cM8?R$=hT#|pcjehix9{S@!(dEdzU z`@A3EeI@Try#JH;KHeYW-TPQ!2z#9^3jQJ z=6d>BUdw%RvH4I&ETgmY;Sp@iJ0C9W{`JF!#%Sh4*TaQJX=fd}$B8JPtuH~(x7$v( z=hKyxKkmH1I+kz&_b~pFvcE)UX)f$eYNkB@M>H2+kNz7Tg}*9%&z6H*F7eeuU#^Qx z>keQ;-+KA39n*c)#I~G4ebuz_2H)?Uh_4>{w5=yb`zu{>Jb6}*VPE`P#=?Er5nOyj zFWr0WV%J7=zn7((eh)n~UjG|TnRxv(k@tB0a~SVO8RyqZ-2Kws7rZ&lwiDoE8L_c< zWZLrO@39d@=z=@@1XKNJ8xP0tqWQ;qia z?aX<2A?tR(XIiA55bIsRRxIKcyQT%Jqt8pbxuc*DZm=Q@8Fu>}?GlmOsPpkazFyM}K_S z{f)N|_muY@_TY1}YS_bjm-PA%m#=>Nu)jHUSZkZ?9na8bTNhb&P8T80!n&5+h%Umu z!&zO#i7j2a$j-^Y#ril{7x_Bnf){^M7b(3sqKmkA{wrO?IVqxx?7ZlFU8I=zcwJ-` z?feZ|wZ|_T-5|s4N1jb{nGrpqJNf)Ea9sQE$6!+27<6Ps@ZQ5X6fq9<6~F~ubn=X~ zQN#cEac+d3{@pky$FR#US_eZclajzHOWU~*=oM&lX)MA6^xoRG| zow2PK+NyxZHjM`^WR)_W(p7KfOwJKxR>4&Q&BWJN4A0hFi?6Sj`lhH}>XXjsgmxv9 z?HU`w|5el{8?WZ4>ePOPn-~CZyB^-V6k4i)mIBa{bodZ-xC=VWg$|pc!z{Dxr1)}d zeyhCfpRxDY9POsr+n=KV+FaWYLQ^oB6fa zrbEcMJ>U&|^-YA9yBV+j;OqbB*WZigot0gOZ7$9qEF(FQky?Ktx#3!Su5|*d#z1SM z_)6}Xgc+lN_voo>k-4w6PB`aqo_mfZ`>+%p9e<&v_u0CCxAslgtB=7{OFP`UHRBZRq;pqH3k;;l4?Tjs@kn9TBkVJu;QuN8XPuAQYu15bB-q4s2g=$kFrR)q_}7jizkA&d7fVS6|Ut<6Z_yumj-{wmG~hnj}?W&78j z@q%#iz7>viuL~Wu_GZ^cgZVtx%;|M;2&&9qj+pKJ~z1WFa)0c&S ztNt@=wAd0X*_$IWpJKSA+qPPJ+@T5WuUtVJG5hmG@cI9z{=xP3GIwNC`FFO~pB=_W zAdx-q`Qe4DfN}S{@U~_rdMayD4SY-XZTal!sq=#eSpy!&U+6_M(fhmXq>YEbdDymf zKe1sAcD7BFlfRJiR+2v0-u?ERaMa$-Jy3JP3pI9S+3tO?;To^xZYW@*kSO7k{;rSup zUm=|eo?%c=j1OIntvh%i!M2gsa0Ws$mTaV@2O@mZ$@A0Xk}D~fwVyb&gKVTt-Rx0g zBQ4`OST0F}H=eCONd3qGPM)gYwd8J_Pd2Y(zqFIc+Va2QZ4LZlYirK` zM%oJfExd60b=X3A%EmB_H8#GE6zb^sD|KKO&V69l#;|zLUxCkMdWkc(Pcr4szbHnS z_Oj~t-aX&&Nj{Lzp=9rOzhnCh+I#(t?8B6_WJAaDCrk}od$+OfHs#ZtL6*1a5!qU{ z0#oC3=vVO1UtzC+M=p2-eU1Ou^M8=8JQjM4*Ok+FAFM0OFZ!(hi>`dn58NlY;`XL$PGLX_&oS2pqvVY# zmvfgZdc)FTW8MBgKi?cG}2dbn;%*Ws<@p~J*We7kSes=ak@zi_y3Xvg6Sy#ojF z;qf;%`tLT^ZP1#m`g&PM)UIqP9g+UOtk|qo`9AvBkk2ss|(n^r@B}(Vs?qO zy!)ehWqYmR%$IDhbEDs4&$7>`uW?X)583Y#eXN% zH@bXp+ut}l0=>o3ib-P~!8VT#x54G>B+n|w@UCbZ|ISYJ6JQ!V-wL955k1IObtUf> zww6-rypy_xi+`zazN85vCj0Do-WsI17QIDUei9Ijkr(+czY+J}@86f5BAE++1lnwk z!ZnThtCEfWT4V_1JxhDPvi138!+$?w!(Rg~{nxLbzq=~(ehG7^{7;+am;d4QJEf=A zV<)!XPrQDo?3?93W^Zm{%bjkMc(TY__I?0$4aqLQKejh8WMujM4fp_8u@6wn^Aq+( z;EDH{%SMkehZuxckHEM)4j$PVd*a?Z(Mig%Ej97X8$z6ED?B~V-Eq?t=ryr^^PN+P zd#HVT@m}p2*G!66R z{-`*Qd)eD_(>(15ituSla`|)FC#+>Wwv=N_DAjtI*jLK`&^5?A4UzraT<{fSuQCyy z&|8V0A+!{(OZJEB4Een+Q#nsY+5J1{+j#IRogJ)-Acqh0^?O+J^(qcfv! z_G#F+S(bA(vMf45ANAC+UeBJiX+tk}rT3N^R_RxiM!XSd>flVnl5bVjEu6nX?=QO0 zT($~(iP6weVB$-djW6L8{Q4@oP1fG1&)%7meM!hvOU>0AY?~gsQlj$w&9hcEG;atS zTG`u&yIev0Zk!-?LiVQ8h>No9WRb^ex(2;2g_sgWMtk{X#&m3f?UmdWyAzma;x83; zB=yzu-381PfleMIug<3E zyzLNhgWsBc{_Y~~_pV3Zde$-8o14nrnmx`@nfcuBjy<4znqkRLFBe%L#Qt~tp^w)a z$;ffw^CkY*UyWU-gxHDjoN-yio}A1$GJl$>ztwrAKS&vkXNY?Al<&6g;mAT)&&Ss| zmUfKwTX0+0bhkTF`?AofT89;%gB#q$;C=_VqMM5cG|qw;AhNn{F6T-;_@_mwSoDay$CI(5i>2no(+LOw2z4CWk z3BC0+t==$XmFNB`_~urOY~CQ>5cJ<=`S{76#82=9>o0MU)GcQ{p-Fk%@e`%M|i(0VXPhhDR##CD~uKAO6@&u#w5_`+%M zB^&%EjUD#vZP1^8lqbKa!B|!bPb+%dTy~H#E%M>31uxef*G11p{C7pm&5o*19qR>* zb$61^BC?)RM*XWNMuYzMq+?ISF7+(FjIB*y8_<|(Pe}c1S?t);Ab(uXINxs8*;eQE z1Nhw&%Ox_l#QnB%^IRMIrWX4rTx$3;*mq7GY3S}vUl0BDH>D56+S>(Qik>x>h497C zf*%z5%w;adEi?k21`IC6q{1&+`F~E{)6nu~^(T}5*!42bp5x(do%oS81Ali8bB8{P zul_51jEKVrCkgE~g_Gcs@V3}Hwkm50@9aU@v1?Y<)m~ZjJh+A~bJ=H;9<;h{$(2=` zBYxECv-Fl4#z}Qdg_bqmH}r?M=_x$-eDp~?Z;r<^cO~2bo|S(qxORhQ*;V_%dHJR8 zjeSm!btQQ2OCV0qEvCN@Tuuew_`CUf!FK?BTj1n2`Y#xA#WNUp7qB!F$AP(E-+OEK z$#i#R5HqBQbieaP&V$@A(8E}0>}{Bs^j7X#`j<$_7-dgX4m`3v`35ZC&%X0`z?F{*|Tdz#?_r&J?PQe zPl(Om7!Oz9Q%AZ3nwOf>Z*+%W8wpI=hO4WGIuh||GbSZ&4AmLd4Yw1sNA*`?`;v{0 zaqOo4MbvBay@f{qbL>+D;T<2b|2LF+V&50>UE?LYOJ1#eXf)^g+}O>LTkNy4(2j@mv5Fz6^Rly@Dc*YC>%q|zk&J!hGo8OdjDk9%9DXvqzD4iw zE4|{uJT z-$VQa!E*o{{t}p2GhV~~@p4Ih7WOph|6>9R?l9I}4b7;W)?wv;gS<+|Kc=lG2j0h6 zqBD%jZXiCzWgUfC4NV0zSs$;0KKibK7FeUF`iJ|nH4u76W)ky zcmp!x^~j1}!w2-M#6ccr$AVX%RCg}8KgyU&ci3_pwvol0aRS$|{={^Or-kb{TK}vu zS*3iRGA2!YzwZ=$ZP5DoAm4TW5O{g2#BkeqF_Cjpz)Kc(5dzo*ri)i6rbX9P?h^+$ zcTjFT_)_{6JaeP}1%E5?+y}$InEbMJD4v|kh_?She}knC-}Jk}>t7#gymmQh-=kfn zx4>^Yz8zlwxoCdzx>oSg0#A$4*&OPW%!D7nLal?f=!Y%v;CLGS3VEdOs!rA6V(&zC z#NtG%F9qb0y-0at^d{=Pv! z{?+-^)af2;0kJvcE9kn?V}-!kG0sftJ`%y7j?6R`dD`mp&OeE+D%v~(EGJ6L{(9sI zPr_RTgU@FXj+}#`e9Z6UufS&DBen=0--N7K>+mekG&Ab5 z`M*nkpzZkSp)=O%K0D-9oqMmw-r9-p#BV9fd8sR!e>}^-)KQ#YRc zp&$BG8;5vmyxu+dak}K+9^z+6r|KVflCidJC%Kei3I-A^0Oql1N z=WILa&yv1`w9w?CequNJ%Jm;vvL`&uALPBe)bt(tZ8&`qd7mI}bEYSs^GErslOjF} ztDTWpTIHMF#M0XRwBS&j2cxN>z-(&d%;p~pjHadnt(~r>Ap=S;p{^Cw)yQ)>Psf$K zf7?iVj`#2Iu0DmLeR3J{kIQ_o+lfDBj^X=1Xyc!^%LYw9kTLfY7vh7~aJuvXJFY5m z6>6y~78_vPnd|ncO~s7US#RyB>RoXidrDa|;g#YO#JaJ+N6u4bz88ENedf+^x^3TC zWcV(4HuB%dHhgoQ5q?txi=fML;8mQ%S+|ALtAR&*TS?zWf}t%C9n%MyhVSiO>EZ5e4F_biid z@EyCcJ6v+XU#-?G;MLwq1OCLw$J-i-o3C%NnD)9iw*dI!-&L>fG*Fp_xH4ZnsQEoQ z@N?$&;C5xRO8b3@zm>hpNV{ovyPKnRJZHC?7Ra{i;QkuUBRIfEe7oa-qlfy{cJ|Py z4H_9jG@UxA+=wVF8-OMLehATxzh~9n@}!o-<$<>kFInAt*p`i$k2)i;gcz%Nog*T0 zFW*w%E%C=-IoTJnr)ex>_vHu|-=vM7y~jE3>wnIgW@6`XVCO)Gr9RJof+ps#8@5^jVK?YA_9Wk(zxxDohVx1YOboHMaxxU3I46QfCf#nXx%l>5Cj4m5 z)e|<)d49r==RE(TNw>dyFR<8_?z(>cHG)Y}73r@cM1`Kg^UV62IQ{Zh)bVZ(vP zD4$Px-1K!Nwy8#JcEz-B(5}voCXu&}ydA`ju4A34(>YCdtcS%dP@u5WPutaJI-#>UNt5!?^Xt2>X+tl-W= z)`c3@Z0W0sh9|!Y9lr)$P_!|Fb*=}xt8l@?RvF7$S)+TMjww~S#xghf)mo@EEQxzg zS5nSH`mICFd25!B8?`3S7mGHjb+}3$Hx(0qHYmmcgX{9vp<>KXWIW0H*PYM zwm!#t++Zf@9&S5k=UCbuTz56JTc`S{XCk)USUsKCZZ3HsQqLm($Li5}P{n0&$-fKO zI_Xoa&NJMd9J@EU85z;VxxPGm-83R+f1-DK*?MOO=iFGYUcdF+T-;2|jbw1OiF&h$ zF9Gl0>%yiqg|p?A%%RQ=3zLhJn)h@=Lu!d(VaKn4k)P~;u=y@LOw?ze!Ij-@s$BuqhnZ~kN85Q@# zqjVpet@{*1`!`VDAZ~W?-;KGN0F!u>=H*1@an72o+h_612F9=On>b<1oXzlsoC(j(S^T4%+izvgCL>p? zYz05n>jr02n1^nDT31_{E2_7c`X)x}%BH+{+tlcsH?h+TM&;j4d7abSK{*3B;_+m_ zj||F1&vPG^4L%P%>8!cSUGDua)_%HeG z>oUGQ&%5Ss>@0h19F+fuq}h5*)}_!PcalPf_IrKg{TAL8M|Vl|45N#<2lDY~q1{B_ zO2T&Y>AoE%=NJe1c98d+Z-?1ePhWMGaW!(R=u^0wNWE_IdGPJ9W|MFN*V{=dD94yqAtn z_O>CD6d5Ufb(gQ%P%*TkOL=pdJ5xp{dCM;?_RHsEVpioz2RyDEIc7Kbu4EryxK^FQ zbu9KyGB%hGpy@e7ul4@!!Q_p1EV*MK%zP2P+lF4-wLMz5^35jw)+M(NY$82|@7_3k zJD5MR>B#Pu#Mn-uZP9!Q_(}qQs-yDK$~CuNbIupyxZ0jfNl_4fUTvchFsw8birO zA!q=bUORE9iH~FWvT0Z6c{LWtIR{`6?@9ht;wzG8ToXF3glAmd1>7r#NA?djmoyHo z!#nbo*2nuIWT%qT7x;>)N3lr`P#5`Tst(CV5ZAgOx?YGjHU9AQ`Pu_eyMFRwQ*1w; zgbigV>k<1BC$LS*Co4izl!vAyYyF12dj5&$_r7lUpN77&fCWD<|5dcnEV`p_tDwm+ z{p_W`UhY>wpILVA$kpRc|x zl6&PY^iKSJtmho5`0+^Y<4$#CyTSS-<6zrWy6J;G=RF#yF@e7tD448EVQnSNpqZ?8etjp-g&P=9b*Y@Ac=aejb0 ztBHZNiE%aj6;Dp>YVAEbw*=ZMKD!3j#>O8$Y>)qtk!Qz$ zF>R<1(wD_AY8ex^=7DIR81X5@h<}mu#YU&OhxxqrMdr?v#E7pVCf*@p#2+F?{5#o^ z81cxxzC-9_VQ8a+wk~7;Qo6k2vHRg;u9PIN>jx#;H?U*G$K+!-ZI#n*BJED4z1wK- z*3|m>kJHxYm)|;2WSomfzlJv2XfGCvK1e&>WsW_Z6PPYOnnxRoLoa^%0e#XuG_aj# zPU3Dec$zU39ReF%?7y464B~Ct@0yo`=cL9*_&LJb;7TsGZ9$u%GtCe2#slE|I@4J9 zU(llXlb%)VJ7m^6tjupDF7Mz@wIKUGu`+*L^EfvI92!o+$jDDE!FFHvGlF56$^hrt}Bg zLBrTAsZLxT-;YI#W7Prt#4Gf0X4PmN=@Y-Vzh`~H?w9?YzUKU!{-s3w*J00pC-)C0 zV9#|iC*91=M8=j_S34c>dsko#{oDP`isk97pCRYWz-a6hyB4MT#xB2i;0owR{p-Cc zaieEwBrcaHgL6^DKJn=6V1m(BY!deN^xOM;XeU6c%Z6B0>@V(nf%sJvH2a}+M#)k_QH9k@h91lG-KFT-f zfmpzchz0zIZze9UAr|m`4$(p-eX5{u<@B)(oAovD9jDD#ZQKRF08bqqsDW0W2XDIX z)5c@%D4(5P4{qW8?<|SXjA%TB9cLFbj@*rH`N5yY=p0`#o6eUCuR9&gRTpi$!6!Iv za`-e)kAhR;XsuH{u{ju7%OdOB_gM#8FZK|x;g<#7O>Y&{69YNe^s9pUarK3=$*-Q6 zKw1ZBouuJ^#hp*x-1k&ZEWTB9T>ecJ>T?C{mec1lZ1WDgucCE(y?h~JquKUP*+tR< zMS``|iSI`OxN(6aH|5TccNVkZMr@62l2U7$FTHt(dpuhI9=OnWYE*JC*I+zsaPI(VWz*eY3zVtXvuNx z*$;h6wm5+N^MA++cd!n`;HpM8iosA13=P207=@uB3PS@hG)7@)h=W0GFXiqt;WG<+ zu;yb{)5E|UnU7gbk@=X_RL{2t`qx-+Og=-CNBB?Ne1U%*Joj1sSW~~xIJ_jdfH^iF zGQRJyRzXu)bq=d&apLm&iw%Fv9SgiL&*9U-=!BBOfgp zW?0%!I!+$Mv_FrEN8d z&bGx9i9LoreF;28JZ3*MQj=&Bd-5&*|BC+({&(_!2mcND@2Y~@uW3Hb2Osmm%XQ#q zF5@ris<&*1J zjPHtX9fxnFQs%SxZvk!IM;-(E%bW)8!Eq~c&~DD36cOX8f%kj%r}-QoJ-9D4oPA?+ zv@Mf=nfBfEqZN97`~D>Z3zj&n@|wiuMdsMPDgPg9XC5DAbuRw%&IB@(g(PGF0-8wx z$%I7^%z|Jh2`VJG5Yl>Edz%1VvS4jo+M;46fD#BQBUo$bZ35KW%wQ_EpzyoBmjJgG zK`o2g^_C=P&4Mh7m>`(n_c`ypSrE0ozuzD8nfE>KS)TKp=RD^*&v~Bbj9*i?f-w}o zxMJQ#A9*GI-T^=OT;Bf8P=5{ly@s*4d`%m)o_VJEs#(iEfTllhEgK!Af8poouH)pN zeUnM^V_DM%(7Y{(U*p&2x^d0l&2{5N=E|3?8y~_Kt+^m_#2>&b>wnB&KC^C&(CY?g zu07u1y5SA38{Xi$QGMCEanWY38z+F{0oIMrE2q~DWC;&Hsbk=}@g`;EyQ~=w@oNgM z87jDDsNkAmux6a&yzni?-GMKg=qp}jJU0(ZtnC1f6!rrR&_Xly{rbU~-G5?zDYnOT zEoQzuw%U9h_7R5;K3Liv{%^JgUxh9TpbL?!)7U>2KW96e3_X158niNmH7Jt#G>mx_ z#r%qf-@tS7On#$z4YH|I%cjIa!basI@GobtvVTcKMlFU1F`h-{xv9A~c#^q4mw8tT z?_u4J`*@|&YkJ34_95I0Y4IMBHIqi;*D5@W^J4jC$i7zWrYIwOU>9|DY@O#9KC=rx z6GIt2-v{IwK^_P5L}>GX%=ZD`qUBqXI@0-ze1G5*iH5{Jq+)_)zNST$XGXaV@T1H9c1{%&JUxYgeapVqoM zk)<86rmWX6&9@o(df|8b*M}O`sm9CpGqXhp!8&k;d1UFCzVj~UM9dLcE55;V31uuD zj3Q0u&xhaX&;NDFk1Qtn@3>NazP(5O!TS#*U@(@l=hyZ#(C!qGHN=mQ{YW)5IdIR= z28@TltPb{P#O40NI`&XUZ=N~=?O;R3K7hVCmfJQd=&d!B^JTm8$@(ed+5sJ~=kIC5 zHf#&|T3L6{kL{qmZ^XNwj*x%B`+p<5WGmY?*%NRl3HOb27V!|eCcEgF6l!tr(YeZ~ zEd&2)=**LbIJ+F&aaSJ_eReJV&%eIDE*#vU*ElQoxk;Sw#PPjUg@}FGkxsk)5NGXc z!PF7!7tCJ(GoS~rM3PM#(;; z`A>Uy-v4Li|B&@pV9!?eu0^zI(Y)BbwIb_?E%`&t1<{SQ5O1rG?inX3m(0J=>^;=e zYnOe|5c)rq{)Zwz%Xk}s>mzRNeu)^N4hMF1@YgN0QyQFmB?I>6-(>!~sO!>0w@}B; z;PNKqyZ$|n?8$4ey{Liayo;6ZK4KW*Bek=Jx}@ICj1@RNlU#Bg;}{n}$NLOpURc9e z%l_d2@$^bqgJdlbU4yIzC9FZR7A#`@khS3HFnA4pk$s!AA#&6YX-CE$f5shRdm;Q# z^pzV(m-^o%zpS%Y>XTb|8nFBoz9TxG!8%uafPHjNGvlnLsUG8qT4wVbStZ1~R@?*8 z2Wt>UOD|*TWYAsqq!{@0d9U?-3*Wv~&DZ%_f$c;d{2I6(fsS2YpbN1DVcRLNF;BTm z8oc+Lt;0}ZM$lShW~?7`l(Tk+o62P7YPUV=UZ=fZzQM|FUFseF_9ATpR2J(uXaY8Y$b3pVCq z>2Ou6>)|d92j_jVW?FOn{GGXao(sLBH#nTDt~2eEN^6`=&TUFL@ zemS4}@g5&yENJ4qw~6!KC*<7oT~+yf^%IHpImCP840Byfc-p#<%#4!@RaWmp<43*4 z#H(9q>XF++mA`R6`g7htn1b!T{a<&La&FMDb-BM~tKm0@L0H4RyR0vPwD{)Cx#(^W zI=j=+|7J~$=N`psd*xW>JGS1#Ip4{0lQwZwcv{k4du1MHmCk7P#S!>GknSLTu5;LQ ziMLR1*egY@q5p;YoqFUw2b_qGw4U;9yf5P2!MoGJ9e^>3)7!CMV7-m2Z?#vZpx3SD zyPP3O-%ino*O2L~K2GkgI@W|;GiRN(ZYQyg>1!Bg-1*>F{Fj`m%AeL+oOv7gdjtHv z0sd|Wf5qVMVD*!U#Gk1I7e!NukN11>oF>m{@*F15aq^t1UY~dz9Opup2V=t0*2b&M z3TcnF_88&YHq(ZjGdd%n(`jL8*_m?%;up zQgt|EMV5+Nn@Rj&e)8>=II%f10y)%^yeiI@3mi#q?hw8Lf0MOI_K(8rTUhHJn4~st zfEVA-9N)NQhL1JN_YwR=^LzR)a_alzvD7#iSb@iO^tmha{X;Buvnb>*P-S96L#;z}o~};LaFJ`% zI&9oUYN&f%_^ViJ{`oue*x%%$Z^>bg^NN<^MBZ3r$Bz>niaz`!c!IfB%!Ho7FXQ7}A$w{nisv^T-`eA&{>lM_@{#QZys$#ZX% zT6U5F~!4l~sW~*tW8Y%&r?$*0v~bVOKWw zk5;~QoHs5lOUo?7o?;s^*u=9*nH7AWXsE1?C7y1Py*rXT%bYQ4S(sa8&121WPG&vH z>!&@7S47djzPX(l@7qHEu1*tr?4M^cMj}(``QIMz8w(#;a%Q&gJIv`<8Zv$Fjy2cU zEwrh-M0j~PP_y~q7n;+Ij+{$?}2RsryrKki^% z8h1kXsv%vS-bd>=&wO#4T3dHeh1X8Eh1YImow*(xfsdv?UH2AyfO_IZw{bVb4-!0? zFPGgfwnjbzBuGs8BgbtcD$o@cg~f572K6XtQ`R)OlPQ|CPJ47f(ml)rNtbuoA4%GW z_SJQr_92{SU|R=lmW@cRmLDQ?KUeTPqr5y*#=YJaUeQ2XA=KN&9;(wGqU&a_TogV= z^}NUWS`fs~N^JE6M{D7=qBoQEYYX|dm33rR4Kb!Y&ht&cA$A#Fp3#B7C_X3#Fcy<{ zUUhYylXfB=RsP$^=PdPPu^y$*V>}CZ4=eFNN2<_SqOxov8?Bn-XI|_m#-7r~Iyted zO}FzJb5_gEEm_PLC%SYaFoF3ZxY2l_?Id7N;*Pjz&K}WW#Z?1`)FXS1G|tD}wCOBW zS*_6KM8j6Qly!6svdr_UJ443Nwgz1bG=2?bmx@eB)M7wAV*P zre!0WjAe|$h37L6LU74OF6j*zTswx@t9Z}7Sfzm z3?0ADyV!|yXNvECz~zH;)sAiD7c}kN3e3*~X8|#k49ZVrJnEaUwWCb^PHY%%Quv+| zr=7MWeKU0XJoCD^0ADJe#c%o$XDp9nUpS@e59kxHQ(*mQ0;WCVIj_!DMNj0Zj49!x zYTrP|ay)19wm;=emV2;|NF42W`Xcpr;iFM}3wsZ2Hl@zTk-@AwYCOoORUdVvSssW9EC< zNUZH*Kb;L9YvpW1{2<*ip51GP?n*D-Y*glumW6*wHn?e^yx`OcPT>>NCQ^r=cX)!Y z9vw)T}?O5a$EwkJgsqD$8tBiBpF|~t!WaHN&bXS94OGoXh!X#oSv|X-Yv=<+ zo@+yvE3N3sOoEo!gB2~rkGhDy$+}P)iftL`w@x$X!^6z86}C}_s_mot_herxOn>)>!CbZK^`97Kn((G@6*_)&?ilgws61<0-HhmQn zs4rX{xu0{BZ%|)fxgf2*Ou6Q(REF*keGCU5_cKNd$XjDK>j(#?6!zd>v)j7-)LmB5 zhAeJOE9ZGC_-&<(lm6dtx7DVi4|YKV*$&etZSM$Z%j1Y>=YJ6DB&RWScvWBq!(LoHbMZ^nW>lhw2p&)zb#hJK{MBZMBp_|FF4Uf`Pu zzLRMum9=3K=}D|dn%A&SDR_?TWl|_JiSK%?g09P=IAe8-e^_`x+liBB>GClPBW&t= z$@fLNr8dT1_c=bG{LiO6*L z5;hd7nA*f7&z#D+@%4#w(QmbkR=XpzI771WyaYWl{NYhI^XF)2y7=qs`tL{(9gF!q zKSA4i>GZh?`m9Ai&q?r^yv3&NK;OBBJ?P=kbm@0L&q)mYjdbE4?mo?W>UlXdUB*u0 zsmT6spdP)ucn&gG9(zZ0nRZhzgzUS0Rr8I=YopQi;}Z(s692dFEK(cAr(OIP+4pW& zu35gXl&Fm&qlk%K1Qou$_lk(<9pnF79dRjDj=OxxAEINuV=E^%dwZhHGF&um{? z6}Cj!TP2cL#!2?8`diB^U=SOrks5}vw4u`*X8UCA7v3uOG3zvSqfUFvdUs{_!7uqx z0-0r-Z_qfV%;8na@P%9H-fZqE4(2Jj5k0l2$$T~CAL9TRww_b<|Q^yWE^h>ps<12T#`Acygb zZt-;!MuJ9gTi- z^9Ofa_2vhMb1r>w7J5u}+G>eO#6HS;(S*#QwZz`m=Q+qh_tfKc<5J z>FetLgMDZ=XHhP612?cI7k>K*eE0u^tMms~8w#(f z(<|%ExR1T|Fjx z5t;v-XU>8>Dym`$vN!VYS(p4@G|slOo~m6#H=TkGiv#Ag zoI+0^-{!J6kp3nq+gZ7bD{Yx;m*Aqs!Cu6%ziuDx=c-|~#ja2-^K$0~{Kgho{P5vT zxfgFb_J&XSzI8G0n55OGCY_b~gr7-1bQiw72wV7)8CNgN{fUanou!5^cI-?1EqmH2 z(DTSx&K)=tXtNIu?t>(r-h$oyo}5&X{N2jh`{5dLuVN} z|GE*ZC&E7|!~bJ^YtcRmlHc@Qsg`=Ui`2}Q@P#xf=Qz6F^+7nO^LAj9yQaR%KQg`# z*?)IqUgEBX`^0W5Hqe9I#NA#2*QMU;Hx9>UBbFHStZnDXulL{zy*+gNuf_c~2cjd> z>&8ZD2VFVs{P;reGua+>7TrjZ%!z5>Qhdrb!w1+)2SlIZ#NW(CUVR_hfIXC~+c-bP zg%8+6iL4=esL0FqP)kS;Lk4j$*9PvRMD}-Fws&$2+&gjZ@tM67=N>84mlH$GD0HGC z=l_X*3#9tXj;od*o5$F@jU->P+)qhfcTTv^ z#j}iu;9}tUnw&uyH6FxI?B~(#%=cE;A^$vu6E3>B@UCWLsmRibO!m$`_A))cWUu$a zqs9W!wH;^w-{QXc!Trb&dzKrOi)eodd+cJ))8|rdF|?-jPNX|aP1*J!eUkMkDOqKS zZb<8PNUNglM*hXm*Gc>vtFL;^Xc_ABGTvc7=dLv1%tm$;JS{CN%dDr}y_6G~O=KF* zw8TfIN6r^yk0*I2V?Xf*e#!E@g1q-ou9tG61K$Il760ESEoz@y&`Y24s5|O0V?l43 zdAC#4H;n}%A4Lr{(^km$AE*UZd+V3K0ndu=5;)cqUoc{5#G!iQ+HLEMYipN4yTFqr zvSpZ|vIO^5xq5VaF6OzwzD8gNZt-2}-LE)f==a?gI|^rm2VI)8t4(kT4%}s;w@@6n`G-7v99)tWazvIJ3&Ke79*!+pTnnP6Buu z=!4`F7{zW=OHSOsBrJ$FmZVW+=@ZO_w#J4qx z+@m%`#bh4|tzX$jf@a*Zuj`> z83Um`&XId6I47uLZ?l>9L{BuI`5|(m@NJRtB<~j15D$4HOP6M9-4gU=BW(rwbIHHw zDs`{SvCUU`o+?J?JKwve?pFA9J@+2FNFP?ZIP(qW-0|_OleEkFqsv|U@axcARF~++ zt|cDwY4lvZ#*}TX#*|vo^PT~Atuv<1eZUt%Ju+7o@od$>+}yT>_dM*8#5Xq&cwhuY z*4(t_piR$X`ogy`=5aB4howAYFBVtLddhmfL&~qv-%sf8ud{~AT(0Hp2>L!6jvbKf zU9wf!wi4jG+X$;|;%rKE>%u#7m2{Ry&vSihtp-P}JAhqx_U`8LDD!leF{m)p`v zY+CG6=)ast%3P5(e4;sDw7*~JaPiy2&P3atA{XO3P=cRDHS&Yx{UDUJK4@2ny;Iy` z^2@kuctS?ACzSQa^oEB$PmbivZjfio;3+uRTG|pZ3wUYNBeq4>|yMtZ{wHs^-b{;`X~DO-2X#g zfBgTUuc?fiHKt-$F8ypLP5LQge)*Wnc#a?P&-HQX|3e@D^`GgZ{fo~)Vwoq{v1>Y( zezdaZD8cT)%X%jCVhhsCm+kMLIrMm8cb@V+Tku3-UDv9@O|`X!^=r9H6FyR(JEziv zj9z~o`VijFaYkK#4YCaSp87Om6(A4D8c4jJWvR3$aO42TjleMjy+<>5lE|4xeL8wk zsqbgpn@5@Yd*NfeOW98-D|#$>UK9RUp~PF3=N{5yd9MF7ymAdZNOTpyok9OX)b;X= zzu-n!h}<`%c8e{<41ThX>UwIQC`4XcX07GKJYB|Fc6Sl=UW6ttViVU9i!Ob-TK19M zk;Z+@6$#ZF6OR*Pm3fSP^cZ4a+A8PLu2Jzoru556pExhYuOc+Pkv>WP#P`1)+zC8W zyG=VY=&z@Jx@Q?NeU`OXc(Tsj8SoX!_Z#w+aONmwg+{)}FISgc zpQw47l)YRQ@Q+h*$IG!12WJN!Gh(pk+TFq)OZF#6XeT;Ye?4+UJ@SP3C0lh4g#8)v zwtNzrah@}IZAU}-Z(t)Lb~hr|i(L*5y_^FMwmlTP4hQ^vE@xcUyl-`Q3m?2>%620E z>;(ttzOsb(uDAm41-8n|`R;wd<3dlTz>&y6_XB%2eUUo0fM4B*K3?B%)6-pyJq)(K zOTdfO-bct<53rZgWtvm;9cgz_zY5M%H}y(86ZjVzdDl=I$MGy@oE&Da-A&FX zO1Rk*V#Wgf3S+pFF~rWIi+YPb9yEsW0m1zrXp=jc292S#Z;jz6tYsfFhRuwR#J0H7 z_`O0MC7j1gnT8N+{1Wu|9ihzSu{f-GBXdmdy^!|43aoDfYvA8KerpftV853XZ&bDb z`-i~3MdpB}zXT0?qu>j^Bf42OPg)OhAFwuJcXR-|Yr&z^4<3&S9)&kpvYf=j5**66 zROEd0Tjsv*3OrEYZD*`x&db~ocrDzP4}doec#DA-nXM?D`7XF>)9{+Oei(S0LpW~& zzDD?lRhBpwMY`-X_9^f)fk*Z?7GIfo(c~$qlx>A>1kX=1F3ih}Vf0bsX|hpi3@9sk z4B+I>P?ive&Za%s%=jb7Y{*IF~UT zC|e3|%cjrBkuwBmq9d2Kthpm=li2EAye9$r(6kf} zoGypTsFATT^-p?jU4%{8kRbl`o)#Q4ceAAYKF0PqIPkvF0%J*#7YY7X17iy?TKq_0{9o~-e_(#3a<`P=Snem#_{~r739hYv%%VR( z4IS;v;Q!;?6yHq#W$rK!&WwC{v@eVQtlJZ=I3{wp=;+{l5qf7F&Zr-a_AR3Rxk6*% z=C~k-Z*+{|?B9uu&Kz6A*u2U-sts_b8#>Cx=u_Nf<(b)g;Zr=%gMQXPKL^-j%6R7r z{d2w}<1Ko;ei|61>3OoyfHgk{%%^JceUZJ^W%CIcU(csIdf<~o&G}@`kC6d^eU!k? zJQJGQkIa!u8f#d3Ds9h2wwLy8_=(BdBD_=HS4bPk=Q6%d$~PmYjo@21XLLQlexAIt ze-@r7HZT{1b4~D8Z)eW}KlH>tQ+FX*%gdTi{1iTMk#b_QeO}iyGGL9%L)an+jnDni zq;H`g8Q1^K92Gu$Id7HqtTjnx36HJ84x}uoj}yIG_FJL)?0zx*?i=?8EjE#xV9~+q2lHRPeVw-!Up((7)7U zv+4=%k?yAs*2;7j^^F2%#lQ5Siax~iu6?_xSJtEw@YKHFgMAS(6RG#y?}D!8*%hf0tE&ygl@XYu}d{VW*JPsi0}KjDX3{*&?avOY@L zKALGU_Z_nZe?6v*XwWb0ah_@J4%PVFml(t+_5~I`rwwT5z$lge7O)NMziUAM_d+{; z{kLdkKz{`vm+!MiQGfHf&^{WLeb#S+{TF(eA@F^OT+IAqoiqLFg19#ZwC&;Sz{+zj z2|l#n#OYmeL=W5#TGTWEE@xS1I z;^JM(>3l9T-?zzE(%F0cw}460O=Pei!F}p~(T{HnyeB+a&#(q-+Q+wQU|)BV$J1`& zyYXD_PRY{--gTb7{jTYUA##cMVl=w?_meu`ICEXvLR~HIhGxiIC%tITZOru`T`p$t zrSl#(=~B)-q^&L1T%0=E*9v{&pt!O5M5vanKAKKEGV)Z5EYJAvmipU5GpsR`@-mJ! z+=GQ~+^qi&Qy*JJUf~Nerx_1Dmgl&qC0W)I>dHZvc0N;dWY^!)v|SLU?u#|%d_B5WW&BSjrBtBBV?Zj&$ZSy##i&v zy`qP+d$Lj!(G3Q53yYc4!hZ(tldi;%R|o0v@-^XflDQ^`&T1!ltbWUy@FMRuH*;Qx z%(as7sE5uKd(Zkp&XBMnlzz4`{+iyg*OEG9t@<-{dg7E`qeihGafzKiwsg#CT}EKm zWv~yl%DhgQhT-P=w-{cezhh50O<=gugkc3ZAJisxYF7FGl=9KU>tl?Pv3JyC0N>H| z{L5(HUf}wE5C?rQ!-uT$O;UcC3GH{7TndJS@46)M_T|zZ|l>FR7rPqYPHX(AWm| z9ci@Vea~!1*6KoFSG>#JFTE=z4o?_*gmCl-5!fh1;@@c3aUDd?m9@5SKGTnIGb#Lp#H2f*P8RJpPnd{@ih0Te2={C=y7FC@}ZfJCVG0FMus?l zE&4){Lo!(Fxug0`=Is5y+_-_ zn?GpB=T&$nerQ?k8`Lsc*Izr!e2P{(Hd2?AKPu&+aVx%C%W8X5`vhd00{Fbd#odBFaUdp(GyfIr*LCr0?rQxC>&(3Yu>ka~YZy^1~D4}d#C)1Aad9f@sv zLSQG~&hd}EE+Hf|?vG0S&r!csw_zsRSHpLLHVy)}#e3w;V-Q@?=34i4>Jojf(AbsW=;dBtkHjl$cKE8k za^y^9)ea&b6YG!N)^^<*>@Ou~iY!RDh@~#E{j5+Qt-a<8`hOmJq zpOiYXsADewe6Lh75<{f_3|(|cQg<6Tzns=O+23QY_?f+hjdt0`&G1mC*wl)iPxps+ z@mZ7e^2=kX>b*EVse)Jc)dTQ423<0E4G;31rO0vEvBWGzPOAT`oCIx}d-T?;;Mb!+ zD<^f(Z`qf>hTKrU8GAI&Qzt{yO|7wFgBkq-_A+lcqiUOtq_>5??bGyCk(s^Qv>g+B zIpJ>_C)RwxCed0S_tUOQG;uEWi|oBOB75m|Aa#`5arrv%Ti`j)I^H~56&*)TOO97X zt=Lh>n7shaK=bL_$U}S$op!@{=qMI>I|<)RV%N1#F#NKo`TIxYMQ2d$!sh~6rWIRV zfiagm{`8!6nY3@PkCyZm+})(pIb*b7-w5o@=ztn%Px_u~_5Eqqbk+x}?*iL*rC&Xv z>8r>)BRK!tEPTA#ziw7hr#Hr=&%W;iI3FA1iM9q= z?z#3)A12Wzs z-&=A((yiduk+WwFCp<&bSl|BDCN!Sb6fk*W-sTvSCWyRqpS;kj^Iz$c#)BR8C?HT{u*qnd9-gc-|NWh{Yz;2)4}h3 z_!>Z?K7j@OkKpSdvb5)9X!@F9{(aUw|D(CiKSo{;^10-lPu>>vGlH|le+;(!jwvrZ zOg_=w?4h13#rqH)zR0LG(3bdBSoR&CoZG7d=m0ppWiS_!9VBOo zK5ee_8(B$#Q^urc12`9Z8iD0aYkr)wWTm*kEWxRkhs2l0jH{PH8fzsg0PN-{Ta)j} zYFC*Wm%pNa9$*pKaWeU2en?vb&%6b8nHySOaXEbp7m974oGA|B+-fMYRVXsn;8@7A z)(V~-qB9$-^t$pKZONV^4*5i4_`eNLf5w(jdkz`;WA0=T`9S9O^~5cbJ6c6<{@wc` zQ^Z#^Ap3}qmBi$baqpv#piKGcXz;{X2<$?eB2&8Qi_UB6iw>(k9?>5^xx4rl`fxL6 zZ^RkfZ}XiT&Dx#UKQF-fmyXeK+Ag6lR@*W!1kd*Yr<`L6&E{Y?btaKH5wr{G$MfgI zA~?73F%U-COUBk_+EpJ+12#7&khXxGB0|Z zfp%aQmt5Hn>;mbrQg(nHSROvOo}eAre{$Z-JWM}_e(;KRV3fD)z&_!*cdT98ft^P$ ze10r;VE17M#vbhBpdHvq?7))jBf1)hKOytOvI9%5_@TA~TMaJ67Ng|@?3L&{dpKw} zF0lO+>;70|A@qvGdS2FkANF4pbF$7|o0BDbVmCO=zfo zvJO3R?HcTozs=oULy3F)_vw$-{T)6~PaRusQboUr^JIRt>|1)BUWG04EBO2LsstTN zB$7S0jG6qG5yMN?;uhkV?4h1q#$AzD;#F?R6WeEdEk3zj`*{8qaLi-vuUAeTC!|b8 z*S>F$3Al(aA+VpomUt_7JUv8xjbr6bRpt9P@ZPxcu5M(dvn9-Z;*#y$YX3&};*sjb zX7oYDm@+TszqGtf`-e~B8Qh;)<#lSFBYb)+u?GZ4W#C9)k-KzHgQNGs(RT1yX++j4 zBeJV)y0=c?6I{8$RR=L98w`7`+++AIaAG&LL*Q(it&08~{MV&}Ldh?_ao!5xT!q!7l7{w1N?-G?uK8Dk*uw8=AdJz83lDY(lvhG_gOQBk!P z%z5~8T~D3KJ-tO`Hd}2ezHQ>K-@sfP=&S$#E%t%P<@oZ*9`18vAK*Kjf0ePII9W|w z{A~VAUA+UNZ;MT zeWdda`R~OiqM{l(n;1=J_oF{79ghDwHvEw@c&44{v?1lZ@Y7PcvxIN&6IXzB>8Ubc z5M6s|n3;a}R1>y&q$kMzMZ{|gGZKhjkU;DNN5Bu?mv|IHAJ6ypVt%LMe!x1^!MBN| zFU41>0)KudzEzWhK9yU+LoL2lPJF8**4<>{Du{pg3DWWN_eJ#ict@r!#iw8Vhn(yc zAJ@K=kpUSi>BmN3TEp+p@Fgd{R?icsqCVcxcbQ2XlSRnef^WZ!kHnlPSgY~fY~nF1 zX&y4Uj0w2di@rI5@pAAR$-Yj?8H@|%vSeH&Z#8+&vz8o?I2GWmX*}x!{c{3~z@)!T zF~8kv^sgTVn2@M=pbRXAyrptMi!WNAq%=X`LH9yXI9p z18vA5MzRy%MPiu1yBzq3Xt?m<5xrykggkc(vHjX;r+tFmeR(@}+EJs;c09rkw`G)gQTBO?92oB}F06&|@@vV`VF>_D%771T}LVRuT z(ZN5b=y`p|T}0PuV#_@5jI4beADcZnQ?{MVs89SodhX}>v8U}SdB3;lG(7wa^c``v z8vcD9hb2n;5MB$NpBX^sPeSL@pz~??4o^*t)^xsvF`opD3*Rwg8APX@48{);9<*N5 zd2~S5dx^giaa~keQ_hrHp=obNZ_zR6{ch%%z@m6R1uVY@7R!&h1AkSKou>fH?}M<& z+?nc-1 z;6`&EM9TOxN9HjPWF~imHR{E73r-r*msbX}*{)+aNh(6*c@==7F; zS;EHLNTNM6-KJwT@VtGMx}mRZE@fq{(CH=p>5EA}fjkmrm3P^E5#ES)U(=_G$W_X=AO=om+x>Pg9?3G|Ajxy(%na4|KVXRrX_ZWncY1b=&*z2X)|2 zs_&u*^3v7;yi{~K2hj)0`H`F>%btXKb~}lwMl7+ePJ9?8UZG>0@|Onpn36|1JUy8S zW~{HvV?hV^;JZctCFeOc#D8wDqWo~`om*4bVKm&^f#2r=V^_w3-rh3~BSHJ``^VnM z{hipqR!5doQY%`9v6>me+^3ZagAO)x(1L7<|7eeL9@nJPeq*C--*tp7`2H-4fGV#>n#Nw9@j(BXK@; zIajoA{&2ac)wIb_jCTsO@Uz!H9U*1=V0?I-Zwu-F35&N*Pv!?7K?aF$1VrWvY zlXWDFeA&tB?DhC2Kdri7l0J$rN+Pfbov=pfJNi209szU$sk9~cX-l6@EV1sZiw~T- z&D8(YrJ#5ebwuYdYXCa}p~QRd7--b*-J6(9aq{4(V3#L3M2J-O;EdpwD~ zwgWlkXnXmh%(>85GJA+U#Br>K4+;+k58O#^mRmGZk6rrxUswJIh8lhc-$fr_#iezj zYb<$;u~%n~mJXQrR=iNrgG#^A-=Iq!?mGeOvfs4g$;-H=0`Z1ptp!uOP?XNtcH=d3#2 z4UT`nenIrH;yWO)oIr*iBKYGTSeM2_Lg1&sEWRxR^L_FfG3@!CO!~2*>c!tORzg2| ztQv;<)=WB)6W0ih-IZvHF5pUQsDgeie@ zzR&8DTcEYS!pj9817Uv+*o9W^1mD%1H|7AN%zc5+92;~2^lhKc7aO=$#wG8w@!6$b z?HAAd>|n0cP*--0IxBmxc={}|vc&qVAzr%!Iat!h^Db+x#03ypPhxANee}pvQJaN!za#t0Iop+Voi33%)Ob^$SDH9f$*I6nb;#V(?QjMLAHet)ZJTV47o{@L(=GI+oOaP$CoJ6Js7R^IQ&wqXWv4dhd9-o2N4wQa$< zmDn*bFOJ*dx*FM&Nn0(%OEzPkK;QJmuqV{k*o;H1=6X4Jks!dp1Psj|8+ zJyXa&aoG&wlDGfD<3G36=I_8BLiY`R@hE-6KceR)|F9n+tTjFVag}@-eJz^*jtRd)G-qr zokIsCXTw(dVrXUt=@%B}`X#P|*j;G4AfHJWpViHZj#zkU24&A3@5h(_JbYk@-g~ls zB?PXf-jA2yCqds{2(~Hhjg<7zT>r00GZ+)08M*(p8X9V$-!kWad&uO)chK+8iACHK z#yHD-Lry4)h0kv2pEuicOJalbCN?ma_JoHk!#mqaKJ*3fWbOfC&5DC(Ooy&spgnUQ znP+!D=e-#`i;cL*{MQ4M$-4xXv4Mwx>%*^dZ-|~d{dR`d-6EI!sCTR^e*49uje4I0g!G0#fFU3dJq{rC6Z|BRMkqLpPxoblF z<}Pnn@RCYB?6C?Zc6vGg`PZ-I9&=Zvdz$>)D&wZDsWV}U)%R4lvxbOln`*?ePq0`RHYHqJ&b&A$q#+Y`sXH|yPJ6KgYps2UGnvnA+IhIsj_Oo zr^H0L5)V2F)FSq-{{}6VF$Q8!Aab{ii^y^vy*6!MAadzj z^O0?C|1ab=GiJV=(PaalO>{GN5If>IV*H_hnv4A8lyytw`6KbZ{AKs;3jgSlV<+ho z@oRVc#|Jj8xk>GE6W@xsV`t^u$j=;0+Khe=eU zoyVKy%ViwW$P@eNwPv}7ET8bI@uY3IlwqbF&eC$eOY&XHHq%@)eYw!jXvuf+mJIPd z)O%`II{e2GFc@>;KUSLJUDCEQZ@w_*oju`5FMjqB@QRann)fo7|BKquO`Rf#9^kp& zaBR!{8ahgN;C}vv#}s?iS@+lFUNrpZ$2xqYN}IDfRu1uXov`^5rh1pXpGOQd=;1gv z8mH_}B%Zv%=3d8q)v@)s#}b>LBkN+C|J=fTmao|&4&Bzcrf}F6|Gh2Tff*IO_UJt; z8b7$l*8IUUv!`ro;XX`>fVQ3_U*`=Y z+zr)3ni{HyHZ>Se&MK*Xl;30gR`a7wcDz4~GR45xXaYwRweTnXh$ zDCeS_i*hc?xuo37Nq+A&-r3&Vnpxi5)%+gg_b9(r{8sW?!LOR1ch<=}jh>U?w%zBA z?w8GT?1aFxtjR6lNODq~|oX+jDemBUQbot|gna8>(j~HE`cNXI2g7 zd@AgeF}HKqFudSyc)@Kqd3M3ahG(-kY%+$;>_BgMH@t!I*0HRSM`uVK&H2Qkrd*oL z2jbQ#iNOPG9e2_uaeahGCjA^n8gEL z6&?nxCG@wGyz?wL?_zKEf?21H^Ql2~G7kSpoz3*Cle~R(zQBIlf(;(k1zx|j9{6j+ zWa=UByk#bjB|eA9f2{oyn_ivmu=nHQ^^bdth>ad2dAviRi`Ao=+QE1GHp9Fta`gye z^WBuIp*fOq;=bbx-S0*%lvOmn%!cS!Xc5D}P`OqY@yxkVOYw{|yy!~pkeCTMi zynRg5#g&o17JfOD|D#oY0yH(TdbWn)u3#M_NE=-}Ti|FJ%X3ooYz<2&&*74e|4%H> zQPs200#%?i#QP8hb@CC$aVR&hSKivQeb zb)-s#Xd2v#T-3tq!auO+59nna;H#89!3owyp{4)kU&fHQj(xP$4nGTlpGjYnn9uJp zp9k`Exv#VRpWx{y;OSND2THkLqK~IDPq48hCcfG+gmL*h^W#0{!>9D68yfjDZj$axlL` zm-(x`Qyy&$@^Xd0RQmuv&YnY;pCjer313vcn6&_2Uvywu z%C1~^L>oMN|KECb`Nfof2fRH>`;D|;6Ko&8pylD3|LNUPMS0_Yh+R&cwOr&jOGdjT zUu~5B+=@(`y>eVd&k~Pcp3l)AS>uK05*T-*jJ2$fVvB$*Vy=(i+(FwN(66k6Ylo{N zbLmq-S?R7x{4^~Y5fx(3u?1XC1ea33n}1|-t*i567j}^GD8|O6nf{60JKv7T{R$q+ zabI(78vDiq&b%sysF(uwKb-gblGKQ+(!}6`oo*FRFu^%Z=s0mbD zD$k8Jm4(uIM+3Ak-(>IpD&KpVFA~4jA$iI(GXKE+z}&qMBk2b?kBjH*dDK|!jqC}Z z8;e->M;2Z!d3Y6gwbDcu_VUiYX}ih{LDz4NK@R@bfqCrPB@V!u8_aZ1ysw({pIhlA zv&{5{c;9Bye`ck>8BA9RI{#1l(r*mbGt#%0K0Rfnch5G{TSoX=NndBB&k3fNB=|N1 z=VPR6_`DKZR(wig0}HIaw2Wn~8WG61-kXVA&UeYP(<%pTNq=ALd+)F3dD6akKQN3K z*vS5(JIH5G#{PT_|6%L}!U~XwjIeDh3K(+N0&MJ3i!A);mDmeD~&hSlB$|mhv$39kgv*>D&$Je{rQ>QM2A1q5+=*U(xtU6P& zR8a|hKa4s>&h7#}_V7oDbGvJ8by#4&^FZBNzhl$VGPlUtXLG2t19*fN2@N>e8)U=# z(UBDWA8b}cPjLCSnsng}x{X4Sxlhu@`<$acf!uu$@<%@V1$a;88%EU3PS%zuki+F% zU*PX#ZISxDHzMa!PR^+dh*$1LpU?sBwH(fV!O4ChTjY@G=zv&bB(Lm6=efRkzpx|; z-7CCiRJMxE-kM%fo6LSA+q1#FIsH%=d#`-sUS9tYc-YS;y*VvyPQ9 zP3Q868Ol%OXNOh(AEZH{zl-3581`CL?jEH!$~c{59N)hyr{@$hf#9(nd_^(#XBZ>w z8II&2f9kOWuQp^&p%Kx?RG|lG22Zx~@?Gqed@Fi;XP_hKTFaVLOy0%F$_3=BMJ}~* zrjg6~zS179WgMX`Vm0^4r81r}C)#N91m_`hSU-nA6Ytu`r&T}~RnWyMHDqQNbTNlD z71>>t8`B~5Sx&6u!+4}!&-GXjgunSro`AtQGQEI}EWSUo$sz)_M4?{au+wJf#* z8B}y(`M@%kaeSvtkEOj9t*C*7~Ey4B$8ZZ2WHpC9810Rl5a4K{!{r-pc{}e>)bhIQX|- zy|;AJj^@w!)F2<4eSnvq1aFVS!7m(K$E+z& z&FtX23%rXg*p8i*$bv1{H?0Y#TXNv-$Q{YFFYO$_uCC4I>^jHTU?)-J#hz2-!JEl* zKj&$Q$?^+E~OPEneF)V==SOFL$ve5p{nOcDI!-6`-kL8k*QAK(l1j=3};E}9F%72V&5fg zkWASyQp<=@R=Ooqs!>{|bXw__Oj$BY=ePHz2W3h%M$43!ZVuLO$&~o;icI-2=~~8( z8)C|u$byTIS?V7XUN`tmYqSCWxit@57HedV7YR6bmn|3zO~ ze>+>Ov=&o#-DJHF*aHo<(psdPQ=dB-R$5S=l|E~Dn@1r7#|L_N|1#egCvxno*Qg!g z$gu?zn48G42l>xNj?GQwzC9znYd`;@&z>i11vPF4`Q%OROXVBf%jbo=mLqfAWK?dM zz&$Rs$vimY-kUA^yV|MP2~3OeEOWphwp2#8@Qc$T8S;@x$%eUo_Hi=p{y?j$%pE`!`$vPhORyS zTcRZr)8;G?-gHI{ITUS+s#WlUXf<^0uB$yy?SA&z!e^h| zeQ&WE)>VRk?$cp^NZfIB#ORmyshITADVcRCN9w*}4`Ztxug^=9xZ|+_Uh&Z*yEpCp zbjh(dz+W%1nm>m2+PJ?FUVlb4TWR&DHpEG74xl)JI-wA7CTvg z(V-XJj6dr(_DRK~K0mzZ{2(UpIdah$<8G;5x2(|6szhd+hbQsz6k!v?#N0Qg)L z7FWBGd)cGIqHCisR2_S6+>yHHSktGdVYRs9b{3Y4rHr+`1lo2R|Rue z+Lk*<#*#jQz7M7EA8{vRC-$@L*ct4>24B8~p-%|YPbk-N0?z&2^FO^n0>x)kJ zUZwL_^rhbz%B)z^ATj}U5bb1T*vu=FNOg}>x&2-PzI(;GO$X(c# zCc)Qj@O2w@4UPb^keR-SdV^_?lBVk{8RJWaPge6T_HhI6C2_*f{rH2Y!K2egsGa+` zlQ0ZEJrO>A5?%|<7pybF3f2XA?J#)l6!_4(tbSg*4qm$sUb}Aj{BSS4wqAHGdck+m zWw_yo;rNABz;BVUYVU^Eeh*$-0pDG^+_|Z2`Ky~IU7BCp5$YX|?Nz~KBeG!a_}A-1 z2Aq)mdR+%TMb{V+ogQeWV0p;GDtKxNG*&!bRd&HsZ->rLKU$v{4NvvKQ;Ro8RZhKb zec}SjPfPbD7W<6K)6>=`et()TQFL_0smN>44=|n4=i=OvRl%7TbXMVn=7g^9BwskZ zw+r5T`|=?Rqv5@>zQn+Ld*7N`c`fy?r~c^l^@%a?-WTD$3f`;Wy~T#AD4FI>+|TzV zGgc;szbigDMiN{j_w;q}EvOhwGpRf#22qy#lUy5RV#o$ z#{4&UuWuM%5y{*Xo3|!(CQm%<+3?2Z@rO`&7EHU*v*9?l%ww-xQ8x^p{C)8CB7E7W z`Eo>;_@jHAzfTNnj2f-rC$7>-nMpQm*}>;==A@$DmgO&QQWuvTTPOS&zlqcE8kyfF zKTh_&4Bd3RXZw}^*=Ijk!+2eDDeqW^Jt|Gc_C3zNL_cT;Hu0CqCvD)jn~kCPk2wPG zA%n~c+FqW!omk}iP5a9LF~@GD&tBS=^>aRBf0B5XN%)zx0o!Yip|$Jak&j`sA^b9* z=XI>N>-MUcN$@G)_f>HgkR!Y}2#druL00Ua%U+kBo2%voWjyX)H|HAZ_>ewgrSGuj zTCy*h^dFIK&b5?5bL|ByeNjr^Tw|j>Xs-P`?-#!NAO0)FhHu1nuY>)_tMES2J+I-v zgS9H#7}~W4+UyA7c?dp;q#q!kPD95hd7X?KvcVv`KZ$vM;k&N6RB456Po>g?KV$UdcvLfTm zK*nZoykmwUjt>!AV$JmO^lI%}D4)U~a>SUq9KX^!B81=qOOjJ4Ve}{Vx z#g?g!HcnvYT*E#>UA6+OMop}YBKKa{mZ(~M2uLx(?@Kg6Y$;pa9&<7}^tbwAF2 zXN`68U|jCC#v2)2=)09NyGgU`(p&MhcPLL*(q2_L4}9&3P-kD|f9@<5D|^HW`Z#x% z2UWKCyQyW~5Y>~$SUaKl8f>1{OjXM?k9uH)Pi!@6r`oWGbkwdXQOjx??fxYytgFnV zg}8jaucZ#^+n{CJWg~oR8kHY^y+ifXgTGeKTFQA>+5P4Ciuu7qvEkI~XrAGymGcBS z^VPlxw2l3gE>}Z2(T|N~UtCRj2RzdOJZ{ozes1^IuUE@frYh|JID2GFthCjYjOIT(Si>6kU$v_pUgK88f0{N*`&2Y; zP)GE8bsF#4esiu|<13>d;!D!FGHBn0&3FgC{VwjNYqKj~#!wmGo8euiO}gxrJ{vp0 zyaywdc$w|$Kt{XmFtG~E*z#{+3(uS(1_=6S=HD9Zo#9!&iIHk}1T;P=Qca(HY0k0R z7?XSc)rhHr7uQaEGV#zQBTMuJ**Ri=5E)njuZ#Id>i8e^**a#Mb<|Kt2W`vw(!0=! zjj_%Bd9sdmGtMx5iITxb9>4qOH*$MMNs7nsh{WfRyvfAM$mN?dvfmaT60x&n4#$Zf z%mMm+fPQ=NjgfPd_4HffOz1wZ@O1xEeI5XBU!T9Ij{ilUFYm96v+y6`Cj(_l%YIgT zMKlhG+hK!N2?83|m-ekOJ{xp>ozNwIO6P9J7vUj%8uu$=%wZ2WRqS}U8w($^rOe+d z?w@IaUsn-#AQ`!4?l2N4i+gtJpK`r4Id__yv#f zy^8y56#S*?-e4>uDNEdhADVdA_t)Idvkl!<74ZsQ<*e7V>yHg2$XTfFZ(IO7`x&J+ zbscl(>&{EL$5Ng6&flQ2I_$0v}c%L%Y{8qs8)pPW7ex&EAwZ!@Ex}7=~s;od1GXI@CPv^emaQ4q-shRT(XKjEO z#%X7hGRyhS{Lkvx@9B23?sZJ`EOSPyW%H1uME_>u=!wGFJm-dbGORv*Ggj+e(q-;# zu&Lc{Y*Hl#L%LOldw8bu3=h4|O<&#gHBI`uL}m3}3;gtVDgBM4zfJV_`Ld;%C%;P! zYx=97D|n{A`uQH7>8pOePi2+T*GA5K+keP2{ngJ8sjNo&f*#qoj(#pJOUuM>EUT2h z`su5azLxPF{+-pnBnRFWC-Gu?+Os{&YKEKr6}`OZ;dGz54+(rM!~V5f))2>3>~m+x zoq(oIv*quWJqud|Xr-FCC1LO!XK8t+e~6lq&w9;0ZrW!@d@7t{;00%wKjp>-rG)&F zH=Mi$A;G-t!>ZA5rg_B2pM4v&YonaJckNfd3U_%wD2a63*0taGl~LL;%eJcZdpTn^ z-!^I1$lE5X@BEDY=h`Y`QYZ7K$elym)bT1jNbliF%Q8DTmqPXcCZ}#k))?n_PgRSNfs!Jvf8MUShg`(cKx{$W-FPk;qt&;+H^Oj=y^R@v??W z97y&O?dali-&MD>hW^UI7~2_RiosQ{1G|oh@CRlzoZ|aOo`Dv+ZdDby+zQo8Z+*Dt}zX_MMBfVpy}6* z$hB4W$(=%z`OtI)Ke^}PB=a;0`j$ObAAIB)2%pGWbM3?qF-CTc0qzyVbg7|_4&W0U z8obMR2p&G<*G|lWQsR1qQSKu0ZIXSIzIRD{A0*F3tz6$Y&J^mWnjfHNY<8(iMQtiwk zN6xj-e{9h2z!8x7d-KF6DTKY% z5MnnCB~E52YbmV9_P z*#^Dil@7&{~s;2V^FR zXc9$45E2c|B%lpJV+r_fB@n&lA<+0}iGU;|YRCXOh@}SG1W?Hg#G--?*8WJKVkBw> zw72#4UJ_855FUnSqCkG%b!LV<3|iaX-|zGL-TpD3+2@@7Tzl=c*IsMwwRHKV(ZGFQ zA7JmxH_kf?z|yw>siC(*?J z65rnCmFosP-wV%Wykakv3+`%taQAZBxC4L0y$yZvM#LbYu^`N(*wDe z*aH`V^EkcVw0S8UbMx!r&QPn8)QG-7(`6*?`|D@!ya`+dPoWP2$DVg|G~gBt{BpU| z_XzNl_$eZn;Edj$D*N(e;(|ld^m|q8%{6M`gt(E|`?=3SA4M;W)??|o`A+w}R|{iz z!ww#fJDYmho@LN2_RTLc6yf=T>3_CCV%oXq|6AuBJQVsbu!+N8rp$SW&|q*>r*F); z;6OTSxz{|M5*pKco|^G@DR7td@1;)(z`+I05WeW*c#GX~dNE-#zaF@1FDrd9G0E^}`ak!;ij4 zyW34l;aYesc#ys63HO9z}b;iHUpNU_u)JP?yn%Ju8qsYan~pAoi@m>{;S(-{9PW-0q*fK5p@>55fQMS|0_hk0Y#) zWxmUz&~f#!!=2lk6v6Kt_}FqFI9B^{sMn1 z@E3aM#jkH$BcZI3@vIRkUp)!@#@}@ecOhR8AL-)Tn!TzypnhVt@r7iGAtZN>E!6L= z``d8WU3WL`$`}N9r2mV-T?^}jIo_;!aM$%TeroU+87FySl}?cr)G%*7`}H$<*!cywfe)GB#jWhkx3D++=zFvD zg>h_-2alysxjeUW4_{y*a1{Gny?-7&mj1~%m++XAe8IIL)H#&8gTb{B&IG=0T$^P0 ze+<{6bX;5Np~bVHDPCOD;R7A+2_NAD;9tBrwmIHShc{n}4tsH|2Dnr+Cte&=z_D`n z!5Z!b3cp-Dw0L4U=Zf+K*1#3k&i8p=9s_TyT9Z;5pzqky36FS;IjC4yUo@3|v@vh- z{L30x4j(lOx#s(`@C(no7&o5k{R2-oUy8=|#nT2iP2H@$6E1w~=J-9~c6{X)#&I^A zLcOiDIflC5k5=t%;Oijre2A`MG~bS;_&7Aq*lp|ad-_{nM-A_g>us*JhuighFOU!2 zv()#&kmZ&7yQSgwAo2tet76d56_qypGaxf7-1ypx%26*bt+Z`b9EV@4h*~;EDXO?$ zDGHT!!CLJJ%QIhgtrnOqJJYol`nVY{7(yTMaWGQXiDXYyRtdh#JnA?N&gaQ9aEs&r z7x*b?1+GhDSU1emeFj-KGr;ZX;CL46W*TQNUwz%^^}BT|o73t?>bjM5>KC6#(c&LU z8D*C}06hkJHLFA9Zr-#?TKwSj=SxWFv<5BLl(!#liN%*b@}P8LeshmxOI{dwK%AU> z@Z~XZzCN}BrqcW7r@)J3vkpX7rt;lnoY~WSJK+aqUk_n_@xDs~2EwCiLqcrL_$N_K zR!5u{j-Ioqfieo??&;5`VDHf}M6Y@dd5K$h zYd|)l2>y(J3%yRdv0!B$wkZjQ>MY^i?oyXVEx04y_Lf>Ve*Y5e)Y(h^#NM<&hPxG{ z>BnA;-@nnP1)RyGjylS}bzwVkF^l(ZNw5|zHiopt!sl(f!-||jbh#quh(L!Mj!rks z*?_z>J4~6d9=N-|59hncgl?waH_`vejAIhxNy87e*cGVuY{iU^FS+Lv=B!4SbE_LB zd#Cc>IAGDrW`p(oXNPCClh&{%?9I zX!JV1rICJ_ckB5#M{_riee&}G*7Gk>M;UcINxoP3E_GZ<9lM5KDf#EU%zqj6Kg+kE z5t6pdE90F~_E~0h*_WAVWuIr}lzowDMz>cDZhz))J^wjxobEbu# zQrCW;g4}XH_yxU?duYgLHJd+vCTIGpW0a4JE0Mvsi=2O$GR)?kpO}+fGtRN@KQ+`( zSYsGAHGRQl)ALf)>G-fIj91r02`(Hq^3VEAZm5U?XR~BJ;n|0pqmv5+FIEjuhSVU> zOP#I^m3fpkrgZs@$0j7?trN()&%KdVZY1smdQg+{(!<2dOQ;#DY}*FDyXQTs7hd4^ zLXXeN`6O!weqXoKLGl#?|o zu*fW#3RvYCta79fX_YZf`;kx1-DG|()cT9{$SdC8@}!| zG^?+?^(5<4{1J-N`X(EIJ?yTY8g9=afMV;tgbk#!{Jd-odZmc7cF zanp_NchZkbopa@UEAoag^xXRg_&#jG)_uI%JfR9YAalSTu%!w*FkARZ);ctKZ$5p{ zuC^A%K_l`FAvSbuPVp%$G!FYKJqA5@nVieP$5%O9&EwwVPx0TmA3wr{+=~gi&R)5{ z#8`N2%(Io-RrTRzs_e@FO^;#+*@lgw;BEspT;NXA?X=~V3AyDNzxMU&I76M80t?#2 zhOS9yg7i;x4ojg8*_WUVam9}uzk@c2-d||L{gxV&^{>_dO$2)lde-S#aWBwj@l0TlBjA^8y~-Hy6GYvsPfQ(U+Q!q z2|9pYDJ%5HR!4e@(1E0a*_0E12`+H8Axr3hTW&6NK;*y3tsJrJXKs5Mx1CUb=TN~_ zm%dire)q(lNY<*Q*ZS)J{3>fX!WijU%Zd707JCX=%YFMLYgzofihaqY_DkWfM3xi^ zY(7D+YW2a-gP&6SbNGWh2h>k+vHm;Qiw;8%67Y*F>!1z$uLAf4k>PxBp(?8u+rkwm ztFn&qKe)6i>o?dP28~#gg)#KQBI_ zojKoP7#I+37!;r=s(-X$a6pV9C?Hxfu=LdoW#N$N3N}$G@VIuhxbg_&D)hzvg#YXT zR`&mI(j(bRZ=?N|=~hR!NB`l4wG~~5tQ*grnWL}Q@knswO@lU7V^4YS2i`s97am;VpFKrz=~4C*FCOiH z-U%KF%~SRL#6AClM}hQL#v}GRoQYQnP5e1@)WiEC>la&`P$#^c_|qFN=O*#fcd79U z4U|6q#Sl5w+x{7$|L|x1^`ZGf*EnfU;#dgYd>gMNGFYLZA`4DKPLB=FmbL76!edH5 z1*fw;^!q8sC3>|yd~nx*TmIOXgmR|X`r+m9oc?P%;9KuOr+x3((yVjP)TK|S&PPWv zPI%@9=&|N&E$SMJo;yI}KRD|3!yXWQmaG8nojx-&XwMAyLrGcr`UJI0DF_sU~^*A>c7Z;SjM8@=~LH6 z4J+^+e%%r50&M5aiDqM98+;!2!NZ0XEDD;Q?}!?f!FP!*kbhLZ2eef~ll>PNrWZW3 zBr9W40QC%+-f#Z@bmzAKGyg@#=^G=;Jo!WV&;Nm^erTJ9ee&U4&Pw!Cqn{dm5Zi_H z-%B3cfWE!GvE5X{kRfyqQFDW3SL~OCpM5v*Nca@sqJ0D zU&H5rbKQFRV8Puu?&WI@qo~8+TuXkj6<>J+KBs%oTA`;sbs|E~PLWR$of78}?^*3R z19x>}lqBx6AY<>Ag^O>`{_I>&8V691FMNvkJb|4kb-H@giA?OG{Y5it)~ipI`w?O< z@gzR#Tfd*Z#Iu*Mmv}b%8}<^t_XuySpzm6@C%OM{im{z$Y-bqTS$Hv#Q*SFsb^H## zQ^u%4_q}lATc0tCZ`?4>IARkU;=F}1y2mbiUL^0Mct6Z}^MC8uza4MB$#c5>AWEDU z8DDviIYe&KUY&-m2X-IZv#_7MS@5GbANg%Mdx7C;VCd#46=%f#R>uuG{~Y18LOX;` zi=IKwbl;|b&PC7Jh<*;e?z7Q9j<&nYMCke{sYfCHS))#y+f02cXp8eULdUmPXt+#aZHQ8Sgm#AAO{%lX3 zcroXIPF=7Vv8;3 zdRgC**h59>yjJ8z7`uI|MtmZCqEqdLapXl9OL>8D2Qbd= zW6z+&8Jf@+&QiY{&SClT-4#z^ARL=vCIUynaCpR-{B@i|Vimk#6mbk2$K$fFaui)k)3 z7RvoHQ?N2LfHJw9A0NUtNn&ACMEWO~cNz;zY4Z`zlNx0--be60kr)}~Ns|j(Xn!wo z_YaO5+d+F`6L!}aQ-b(#X)cxhiFm}N%5^4*rTJTI_1>Tk)ihNTT_C(xH9FJ%hVZuZ z!Ezo(7RLQ-Wl^Abk+p6?|8 z6u!re;jTS($@xIik~f^ZJ#nIhcr6C%oWPg`X65j3zwp59B;#AcJmu!QVb;MIZsGZA z@`r;*f_ow#?cgraZ}C6Qp(jN0h&~JF#5zpZ3H?KkJ<5l_Nn^MsA)vt^cR>^Y3DnL53^>rILH8Rl94UVIJv zipU|{He4dxoXB^%-+=BKI|}z5zL&p-jvK{Kx{7SG+iqOW2V?KDIzHjNqt`>k#EK#x?Vu~P6lcvMUZHD(8y7^D+=5SdDXW)bAC$Rm9qYz}Mrc+E=S%sn!@?5> zw8hho*QWT)oyb>>ZaK2_UB#b-=~nbWzL!1B9zCO{Ki%y+&2=f_gP6A0?ldRLeicWX z3Nm&BbVz7Wief2@p`6#pu!DRiY_!GyuVbG%QRuU*Z}BrMxDdb4?Pu7V&+9YTNnX9o zL3844$_QR~eGT`@i?3nHd$An(^>Qc8iAyNAHdxssvIpXo5i^9bvOa`|7CYKgw_#tn zz~ng3yvvzwoM!EORH>WL3a$t&KUV%cL9?vQ!cRj&I=b521f@uDo9F8^a5Y{vU6;o> zxRv?w4}>;xJ`M;}#s;37yl)ie;Y{v(h#hPGm=#e@&wURww%g6fwfecM@vPj{=zDL$ zgueAr&YoWTP5Q0(U%y8~tn37Ayfd(e&A^UYfkw{6{yX*BipsUD7k~OF_#c36{$kZs zpm1J7)}Y&ni#=_I*nN|KDf1b|xK#4XJF)$~YEi%%x9zvv4ty3eF0lhQ5@*s6*$<+*uO;84 z9}4ZtoU}ljDjpgg9i}z~}(h)F!K3bjI8`KjP`YLtoKprV|EI;2hLCSgU+gZCIuU=+dS-CklSY=)-(K%P?#)3!G#g$F5f1JAEMX`Hryj8bTjc2srDM*hdi`*)1f zMllC6hB(?5J0tndW;$aOd$o*bHgyhUJlh%1CfXatJ}vLpkw+gV`J2!?2+epk(mYhJ zZ=11jE#JNMCD0FVeFsQigb#MjlIK=_CSRpD*rjHu>o^5@GcO%D_51KtGM%LpzqjPZAq~pFOPu_ zfZhw93ywPQ0k#;KvJrgBH(6I!aX(`g_$J>>z()RwanW>?GOr>x7yO9-WN9U~=C74f zW*~P4bGfg37@51^`X)8NW>f2v7L%{--?`t9Jki2jtnjT*lClM#*eYido1hptuQ2s^ zHxOFHJgpem;~oCXtnzM1k9VP*1sTtZ&>rt1ky-MtBCAXEMh5j4y!yYFH_o=06VGoD0&8wsr`mE z^kvJUm2*OaY$%uQfuAY!TJX&vEerV_AfD!3f%|Xy5$PxI zvCJ*G*E09>rwz3#|1IX?iBArAd|rv)M{|4yC;zMba(4FALVO(gvT{%7)kGyf0rKcD|d%0yA-M*idYpTd6z{{{U2l>d`I zOsgF;plIHS!SxfzjCeis#IX8_Sv+Trcq8+qxqjj!q#qgac4m@a(Yz${TbW}8b8^~O1Rr3eU~;?f7_blW;`JEa&O zX_OawB6LF5o7icJ-^i9jzPAaFZ(y&EcP&^cxGwqp*^dS1MRq!be_wun{08s~<`={- zlwSzH2!2EPh4TyJ7s>D5*2T38Cq!qi*`SSnaf3GRR~v%n{jUv{dH-dO$$aFg3z<*w z{Nje7(ZAnd8U1_S?J7H;`76Fl`Vi^=^rXi;bv|z>|EwTPkI4)7VwU| zdL8%BUKwS7LYw!{W=`4HnfKC;;x02!$|pU2F7s{P-9w#}A1&qo^pxat*L9Zk8c#an z>8~^IAGsT+r%z_CXB{1S z>QH7G>uA?gUt}szHD})R)aU5`Kc~+;gO@pFhjjcGd{pG;(bEgBAiACupG7B{q0urY z@n0@lDf6ewZ@8A}4?Qp7EO!8#0nsOwGM_SkGH*9OX+5`VT+E}>{Rx>4EDb%bB^DbKF1r&vjJNm+3ya3pTZvte!d89N?Leu z;nxIzgKLs6qyC3%c;kL>G zHBk1Tf_Sx}a+IN}axSqrhN#ObP1h~0Y#Y10^8Jy|RBB`FmCLDPyBgePWPM9t2lA6P z#b#3YTIqK+KBS~=LozW|^XDagN!xqWfi?^FT#Dk~v{u?y0@_;qmRBAPUtY=i$HqQd zP(XX5m8#0kzAGwIvC{~^Pv+6mXDSE%!d{tCQd0Rx>ge*cEq5g&@adUUVyk@I5YcuJ z|F+S_h>d9zpnbGm&VP%3e^m4)5l+ABl~wiJ30h1Xw;0Yy(szN~abPTIp%>D38H>id zbl{=qp|1hp+!5w$G4e%ZB{RxpeDq@pX_xV8=NyASann8UCbqVVeK{A|o7j{{z8$>-=t6ChfDg&8N4kWpTxkR=`SJB>|khi zh#Tet=j-KsvcyfNy*S^^W2hH-jNHR+9-|doN7bjzd5C)_e?2$bT&?p9?(#vS>Edy@qhp1&*nZb`SZEe+?N(!89%G9&4MSE`_tcO zA9|Sweu<1*>JJRHCO(9p#tPM6x9Ji-VkURN(@TF@83}%ZcPqiwm2x-r*Q^`CLs?6e zq*F?}c?Nmc6BpyDJ4X23Ha9LF_Tb__!t4G|#l>E{?mr0^Q;hJt;9^RE7Z>3h&i2N| z!{DOuyMl{Cr-a{~{5^3IKXm=#qTW8ZC^&co-&Z|w&@SIZ{wTO6c8ky7=j(XxI;B(k zuxz&CDE~LrQSpSq@%$A=hwUC8WW~L03%vfh401oJSnkD3EZ$(}=zhD~qe?+<4D4$6^w_;<>U7b?* zU7Z^7m%&{Hbbt2d^@{!X*i<#+?;$%>+4A;!?v%`%>^O6kwNqkomLPK#y^W@*HuR&8 zW^{$pK01p|1VJGwJ@iT`J#@%QL5bd1-0tVt&Hd@1M#*E7Fy=mzmyo0P%3DUqgx6m)_)yblc$*}VJx z-Mo)PCn$Dfq4*GuM<3V;A8V9n;)5oQ(Gr>;QJD{P-xkr;S3m4?44un{)TGw+sl9cA z+#NUoUn6$zcflvR(C3Nm&kputw_dR0qK-=B!Yz#9PEWor#?+m!TbFeQ`PL%$Ek}3b z)&)iY2e8#y^UimYn~_DGJDZwx>SStCm7k$-5@YOm#Ja7@IIdvnDYwm|u1^9N z^?ApB4&U?PRfgdOzg+TSlo{K6D>B#@;q_eD0;+kBCQ5$EdkQ-wvA6pf`4j1@*g2bx zYMbc5M4qYTGA3lWyBXtk^i|i5k;g!~fi)hZL^Mg?Ri(S%*+={)oZhb5!|@$)c8SFv z!CHyfImVs@E{?}XT&@wnDcjYgt^1Q49S{0BT=?|TqOY>M9yB@5J*YVj&`0-pu&FtT zO--xcgazo5u?KIDX7BOhX|M-RmA0;}lHZH>lNpN-ZI&t5*}eVs-bI+kdeXmTk0BO| z+)q%QxA4v%yFKSX{2_zeChTI4(x+PV2(K#2mmj_`$}W2FhtM^PeR36iZW?ntIX;9xTjciN$lu`B19YJSXdusE zDff;#W)w1qYsf#?f7sA2^Z@nDYl1wd8K!li2N2u8D|l{450G-5Qdr7dA4gW-g`8fc ztl7Z)lSlg5-xv-fJAMec|2D>O9GyWI`hs-c>FqPd!?f3LSNL)KzN4ciceeq0xR|u;NJZ@zG^w!pk=M zsSe>81~EpBHEzmv>GBQH-@~Kp@{MNhM74@v@)9lg z;6*x-!#Pvi6pi=!nDg%Md}eUd*oRgR6s;S^-vkl=lvL%+;wAG-m3n&pyL>;^Q|Xge6oojmaoGwTL>>rTb)h`rI_I?4FEx;}~@ zat(g@lJ%IZRRMMQ=~9ckMWP9`(vPllkdY zE>`l9zgKzssh9tOr(f6Ems0*y%J;9^7ymQvIOuDkac9`e8ri>8<}cmYF}`MiXYVRD z78+R_aY4#ZgtXmzSG{}A-W;mforB!(Q`x&z=FMN`g|?+mnWHy}0~H?>HMYgGkG)B} z>1f%<7?*bsn|`Egwalr*5Muk;6^es1_O``W@VkLuCbq+~73>IzK~9{`{_Fz0{x8kN zif%U|?dx_U=kN&(o=V;5g79zAP$M!vGrIj9$T-QTpWFQxYSYon4n4v=FMB7+uG3TB z8|^Dp?)yPU{)o-ahv2cQ%NhI&O2F3?bZ9%Vw@E4YY1fFEP{XMMjJjx-NO{@05EdN>&PaTB!G3@|%mTg~VCt9$Rvcu`M6v9^27t zzH@8`(buL}jqQ!l{7a4Pm7ZgpUpu~yW@IqFIeBvumovU2o;>%9uPDYhlRPrMXUMN54;yOw>#D3N@IKrN zDNq7^GTDyl&u?NDxNegUiFz4qh2|}Fg!5R>Ju?n z|GsF)MEQ>`*)dTWw|;K)xOH?MdKlI`e^T_Zw|Yqq2{ z(&#Z2Ne7Y^-6}k!IfyhmRjZ^!NTb(mW^XY^kVaRGPqi85;iN~R+m*D1bUgLLdzddL zJ&O7z9Y=Z$^-KC{($`ZzJf-2aj-yCyy=&55K_sbA8`q;H~rNn1%zp?>hx zJel;Z)Gz6D($lD4(zlSlo%+Eq^N&d1P5qLdLHZ}u&pI*RMS2$XOZvy8AE16oXOqsO ze)vZ7{iNqmzoc_X7gE2ZA0qu2^$XweQ_@A$FX=}~FQR_<5c7P}C2rlec_Ha1sbAg~ zlU_>w&{uOA>E+Zf>2lI`>X-B~($7;rw7~o<=@+P9(yK`Siu$1|=3kOtOZ}35k@U;R zf|=t%;7TKM#;by?E2naI7+ZopDU?ZPUQP4!>W&g8?dp*qM9E#aO~^H+KH(JtiTg-i z`z`#cRITj*u&!m#7Mg~gNc(%hZ|y*z%xr%9nHQn2gIV*k*GpYD^Z&=A3s(kdnk{IF zvL~jldTzUMu+wLdwRS+*;#$8Q^~pgy_9qY7@n6Y9cl>Aap~>s!=1i`hdvN&rOuw7g z%`GGS8S*2aK}u~b&oA+u!}Da)?~pD-#^f_7r8WrN_?R?h&$RPhTlcGj+hh**`#y1^ z8n}IQ`iYf8)SxC+8{8B}KeTDep5r`6(-%W(eXw;pMqQ+eDLDaQ|poA|~ zW!rJ%Ab4)A?I66rHb1{ki*Ab2<|ovdek3+h?)`^(`$(zwu|qdDOAN`i+^czfluk$V zXE{%+zQtLu^DdqB46g0*=9TxGdgXP;ui4P!okAYTw~lvW3v%i_S<$NLZdzuN4`{20UR*xkJEo0& zHIi{Uz=L?|kn#tZBNNZ9CX;O>;}o9ZJ>gaOPFcojVw_7WkI_fnu9)^kUs4S1h$jxL zjBmM~Z(!T4CCe*QRHZOOU0zupT3#8gl;ATdK(~W#_M23z_<$$GN>qHn8NXEi!53mB zDn8&2F%lIYrPg1^?WxFZGO*P&>h^zLeD&h;${zUXefQPXJ#pTR$ESHGb0~Ax7w?bp zO=H~seLKWAv6=MROKNiG$B%eoLBs{eH}q2Ri5-iq1%DO^tZCuF4o^Zi-Zh{LSt6fxE2vl<(R8 z?cd%0w*SuYU;mG^|4$fy=y$h|T~MF-|2F^etefvM|EG~>iv7FS*1ezkzjXWkkNx75)HPf*kJe#)~0;OC9}>?R%}QU-unbslwV)kwZZj74ZR$ ze4;kmE;ivJlZk_eNMEE43s@95y_`5VxejfZhU}0SwDW5Yn&-)Git{tQOd+qh>{$lv#P*~B zygEWY8$Rx*GKL3K|E4ycTaEDv(fESAAKdfqdxBg4akTe+3mLcc=_2fQ3a)V2e;>N){xi%x7@1O^9{u-=fvCgeh{?b#0PKv`9$sVFHZD_bE5ARyeU}p z0XR3cUz{`cz&V2l=YX}{gLiELx5GyNCg!-kk?{-NsNmj*)SbdR!9{`Rx8Y)d;39qB z2s{NBfoD7G_bV?v^9C;7C%g%F_mf)57Y97cX~P$IBHM&d5jeW{YwtQO*I|jjr=6a= zD?MQ6!Mjdr=c?P6eu~U%a|Uo+3>+r`$0O9a95_A%9OnYZ4}s%{!0}b!*gyUW3_oy_UD06g^8rUQ_6Hgq+XjRFY49@n%#|eF4 zsNbE`@5*XUxwj-fKhP=SuS#>i+TNQlgD>ptZ*RIyC$^RK+kESNI-KJ~{h{KOtX;??VUDjMb7I3+{{v+r% zvXNVG{>+klb-4lfey&HK`v=;z-b~r>j(~@xY((7{^5lQzH-Sq@A;?|6QIE(;!#}bn zMSV=WA=1ytj(}^WY*d}t*L9+s*jRddN83)VW25noy0+EIwvDBCbhJ%ZIyV~ctZUmS z&v$mTO$y-qU3G2Sc{bi%*EUby-^Kd@^ylu5O~!lbj+83fh{w|T^#otXQS{!&ME)r?6lKVy>2V=?!4<4&RnMt|}pKJzLDDB~N6Q^#G?!uG++wo>LcSDw}J z1^5P;aq1c3AK{xcl>2^>L#&;|5E`P#BH+H91^sdL)yf2K{#fKZBY+`i`mDqFxIoTv zalWg+lrvuUoGQnN3&~o_qpcL|*{&}skJk(q$h0&fX7Tq%ODC90@z1)?I zS0>O0{Okp{Mbn;v_D<%cCCT{P2h&&P;%j`ur=WAS(*H5UX;M;mB}+YuT0F7yLCKqz zl!6{z_%ZMJ!sb{VDaNq{4#qEI%+>Eo%iVYP{crKjEqd~6#7@o-S@5$ZFGT5meZcD2 zPhXm;OYTUY#plE0qG$K?JI-_GTJB>?9Vhwzm7Z^y(~4iBDC+wfn+L%m>f2WCL(B)} z&V=45g*iDWeO&LEgIn>hW>$u_^4+Y)7sSw4iS?YT+kJ>H1b1J>-;t4e$5N+ewyyN2 zjNHMyy<}-rEASn6x78sw78dG`KyEP$+rmb@4tIGyRsdr?0c=}G!{2~6BDceDX%_mJ zkn5*2KA9)#=>R8z*?1FeNPkPI^W=k!gJ-jvSg?aN_#%Du)-N_K0t?CHBf-F{IHi=(W=(--KX)5*S`nmOWiVnbLEl!(Uh)v9l1;L8FXS)wU!$j^W@tx|W55#wB05*D?#CHn55A%rXiSYk>1<(syz`;Y#bEB61IH@h_>~T;;m(HBT{Fbrq8lbFvIcn5 z#49p-)5KnKlt#`s$GT$|IjJ8ZN2inaoV4e^$oib?r5S$iy0BRAL-= z&HUk6HiAz}iSd*NzvdvOlOs4PQDmQvK5?GpTp=-?>gk`$3~ayf8$ZRf9@fixSit(# zY3dCHr|~%@cO%b2W4v?VT@$iy!x;bA^{g-G-(kwgy1f(liT%)NWKi|rvSy{c%#+YU znWJ3VaFG_eBQ*Q`f;-vY?_i(5o&A0W`~Gz7z_O4p4$$R`+BtQa=zHU_A5)^IJl;%S zb-uy3lsn((1{Lg&;k#U8%Za_c=yjd%Z1A^FEy}A4{88xis!;(28h7?g>JzWayYA=3 zy`P!?Df)MC13Lm!pXVt>xpnc=LZ_#wfd#GjqKO~5wDKz8o`Ebp9UXyo{`P%~OVF(V ziz&X=ZSg#V2V3P{7Wa5Js=UWGD5Ui##aS5*GwY^KQ-*rKRr5{Xmy`gBhrI_{^5wGH zc>8j2fVk2D62rAz))swYUusKbk5SNb3mjUE#E*(a292D2@0HkL1l50P;uo)~*v`|R zA&lM9Sm_^WT{%;I{KVm4`pxqZV_d?(bE*3_5hI;*l{0<_e*H)7elDIf3uljneC%Tj z;9(2Y_zL*krhu5Is4zRmAPBrzaTjOxZE zVGorjLRv*r{^7(npN{cYAq$?u z_~O-o-NZ+%5TB0gp}no#1$Ym?!%fhjz;i|WZjDy?!BP9c)ggpG7)onic#6BkvDH;+1~gmub)_Y z`Fkg(to-bRi8#aHZWH(U!QZrf%>N#_=Z~&nW!%Wd%GS2Uud_$!FV;TEUu^a8c-t{cB#`28yV!j2?1oEE97I+)n@k$>boNg-#j%pJ8UA^+c zYE22NsLs#T@j>SOFg8L{n0xk}Pql$l(7%LLIPzVIQ?M0krN0x@@CuoCe{eHH{}z|n z#vUzuQLFI=;^=va99LzuG#=&Y2 z4$3$+PmEQK`EFtUzj`p$(F%^oG0%_Pqc~))D-FulGQJ7?1>O=9XgTxnU4B(^fvd!? z>RH!4$GWzpA78jn`w2Q5^x9L5nk{;y@>=mx?in&B;^`5`v^eW>=JY|@tY;3q{^S(q zbL)R~ZPl2w`}Dr{ADcN8`nTPnZj2#*d5uA}g+cdFINI}i&^^I}-Pc{3?k)Kay0;2g zhj8Xx{0+Jna!It}BRcZK)bY#M9}+JQb? z4ca{o`lCKEWKGZ$d)5pwsJnw0k7^iFQJptS$0;A?(Fb}cGL2HD{?i=xrL@3FwlpQE zVtw9$#2n~hDK;^wirN&d7@0;o==zyqmCDr}IQP!f+K&fj8@Ec8i0r^)a z22z)ex|T7nO^oLhegni#P$Q4&5_C}TOH)IeG|f#1OJvF_G}qq0Zg@8J|{V5TwTB2Aob46(^sX(=>P*5t<#U8}R{gT!&_WUs$> zz0k&Boi zfoZn&CPPyWGW3J0argZTgVy9M{Af*%;>7(0Ls4ty?lpqFM}&(fj8O8 z6s_Wc{Ld17p_e1EqsW0)J_X+H;(bp3<%v7j$rmr48J>Ot_Mn-=2V68K;0u80?g8Q{w! z(TXGA1kV5sY-GN*QP4oumh5v$2;C4BzLbga1+8O$Yao?vD;` zzWo0_9bCHN|4%x&Ywv$A9sCJ%+V^an#C(1qIvB+nW+pPib)1pCGQxCZgu^)ZJl01> zxEL8>{hZhl;sA4Xl2A0lhl&&^B!fpfD4 zKDeG}T+@aADwQ*{Ye2Xy0$Ja8$r0b*)3rL6_Mfflrj<`gx*s{>1F_1g&OUO)6xKvM zx{$9#jyTUPN0d2raV}GyyziXrUW3CcOYA_u73Gm9c5v4;oIZcW8M)qbMix1o$Pvk- z%MsI!ZaLyYqWP>-TQ5WTj zj{G=u9g!7n-0^nH6O(mWVtjn~8j&knxXU5F>a^R#*O-`Bg?SbE;xEC&-txuXeGvKL zVXutm67og4yZ3dQ)zQjYiBqGR9{V@9d~pr(#f5z9K2wJ}A7g(0c3!f(u6vJmZ4sPk zH5Tv9y;55v=j%L=j4>O(fIVf50nE3^5qruQ55+)7|LT@8-p)6Pi*f2UBX%+PgD|SP z-(d0O>@q6$5%arNAK+i+Mf7;w6DPK#+h@eZI25HSc5lSyLucJcd;+Z{%I<;mX%#8)PQ)vV5 z>XTJPsg*c^-q?W5SC&fopIVev7RBUNA8YCZ;}3$7Sl#avPJq>x`g)SU1_pDj)q}&V9?o8T9&L4OVTx3>HNL)9a}?@ z5x+q?Tl#*9I}YCX-n6UhAN77yHqFy-(OJ<~`(JaUpTW*i^z*NLleiuo{l+|W$DTNY z3=~}*_i;5RFy)@V(P?E~Le!2o#fQZX<{-^F5jcqN4(wPw?LEqRoPaD%gC^MMyTmk* zd6oME#XMUtbZwD14c?f95<}^3`gCePe!h`$x7e(XgE<0Ui3dV|X|r!!7y7P`RmLWB zp|ED|+~^+PY3AT1`l8iu??@NgAbP%?-8PfVx!u7!bX`dd3*h2nAKA%Ra*D^=|G*b&lpxBXz>_ygjrJfo{?H8DTH>mD&+jJl?GA_xk+<_mc2w zzHteAybIirxlq81JZN_U@zAJ`~WFijF^RQOFwiT~w_}e010F?mfmL`nUK+K5Gu6 zOKJsg@Dw--{s?|I)XSa*FY#!Mz_vGS(Q#QxvClJe7uJj0JQwiX z{Iu2K{r(`&LdU!`OkcmVQta_EhZ_`!#@M9GtXT zlX8inD039A+vTgyxH79_rmA#K;SNk+{F?i;^k3GB#Q1dMm*%7B@o+V(lH|s113vOK zY!dLtzJU95r3Smu)f%x2PF3xSNob8)dmx~AqT-uc%bKn|P->_}em!yhub?x;q!K#w zf~491bb0};QK5&;+@n$zwRXnJuGQEr?S1aT>Bj8_?tlVI>CO_hH3EMG))7uSk ze#JU>@c!Jqx3I%dvBxmbo{@IN=a74^h=4yg!=K0CH}4ESJP?y-C9^;%5K;74(UuYM- zkus!WbN=W=f!l8kLu_B_I;fDgYtca+;C|DP!v%#%{-yAi#N$%EiXSb<3 ziaU79kL%VhO?AX_SADV)h%PQ*x36;Z>uHMrZuY6&OVFQ9djAi*(~5UI2M*ir=Vz&} z-(B+d&fTTOe}KdqY?Bq=-PTR@yJru#?k(l}toIvsrzwihn}>P;4st%PRDj+2F3i z+@-S5sO%M11zw1;q_U>e0m75;B~#&d)S)~F z@l5>eRP2J)FrGtrj^a6j=O~^dcpky?aGpo-JY4A%+<5}NQt(OOC^+>hI3@UW41A(L zwT3~dwP}HBE&Zv@C2ubIE~Sf{ub-(RE}p6S#*7wy{<(R(bUz&u<7~!@ZdxU@GlO*f z5M|4q(ERkSZX1c2S1RpaVIvVj{9t|G)pdrU&RFPHhbQiWlQosk8gsJ74ztDzc($>| zrn1I1vc|4vjcsC0ZDNgG%^Is^jVr}Uj37u zjJ}uk`L>I-coS>!ctDywKLQB_0|cf0=5J|NQZl2}Tn(Y^IBP&W&p>te&pdWrR<-iTw{>b9YW+ zKTKm^bmOhE`2zQ@Bz9~6`=krL34cJ2FxDt>DK{=b7So8$M>uO5Tbf>WOkM0>t@zJ1 z`%ckq-4fVid>^=#U+k@m^_mwn`87hcJT&196w zT(ODTMxK9Y4?NVp2a0`mRNI1mA+&KR{lDyiFMQCodJXML{5@U&4_tNrgn6GvS{d4p z{vZCs#Jbm5_x{LBunB7t8@UAHJe`3ic=i8roDs!`ZME-x`Z^W;{|=A-|2M4pdC-90 zKm(3L12(h9pJk0-%NqZZHQr8H3v{5Gch%4V3v^%>WlEqEsmtC!v3A+969=hVV#FU{ zon|j{+YE?ph}aCId29x5^5Ex_18=gu66)J@WtP=(S{c0imxiF-huAZBv2J&FFKs?%u^Z4cY$5?k`w3&l?8Vc5Qlhw_oX>pWE``$GhKE{A|CZ z?l0J)U~Dc@Pd8uzhXVME?9Y( zI8~`)^Yw+;*P3*DubRpWtFNW4E~A{+y5nXE-z&bkEuJ%I9DB1XK;_<+V*ep$Z1UTl74Sv{GJ=`=-kUh~r9`ba*??GfCO(`QG!1ekS=IZNnG#+qT$XiTpu< zAO3*xh|EiLl4tH5Kp*|!?}M}2jN9GlSJV_yafb(68@yX=T^jpE!UiOobGKM}f zujbM}Br8G16J4W`7sQF|s~)*j1M(=5ef5?%=yD|L+C0pCR@2Xgv7D`S+kjx_8eo&b znO5XT=NBOFD)q?a8peNX9|!d4c7KoqoRfG+cb-r>h? zyj|m5DtEHPm!#0!V0fUKCC}_lZ~e`QxCwte;o5cb#F{6xH9y0C9;drsCQb6QCqws% zZNF0Ee<}R8A8RE+4Rf|3i>P>_t#PqxYHQ`5d^UIFQt&qxkDYp}L7QOaZoUawQG&Yo zb$8y47xOl7FMbg9V3#@sKX!!%kttDsTy$g9{uJXU8gj`7{eD5XQ*0sM{i(?Nuu0VS zZ>6(lNm`Qls}%WVDn8H>cW&>9Kt`u=$I)ffreC)6K^wn+wjpVMn$r3450uXR8A|70cc@8?yHbwr~nDpxn_La34R)>4)OZL=P(4)RvNuNXK zBY73yNwwJnjJ1mIWd42lPvu|bKaGC_|JGWm8y>5+SZ_bv>16!xl0LuS9@fC!tcAN+ z6L+#U?m$LyJA0)2E}Bl~tU>mhSwEH@^49boeIX9!syJ|xb@Dg&NBR;7{6$7(;^&ps zUe}}E5Yn>u*xAe6-xd3AhkHL-#=9H!H7!2b2h#6B^nWnp2x2@#SkpsU(?0q-HsLd` z6+Is~_S&Qb&XvTtYei=xYp@=lkMAX*H-@h@pK~;dthR)8qVodsUHmu?f5u~e*ZxFp|mwXXM`41q{*Etr8AGUmO}rESw9u5z4VFh zd={QFQmvg8U0qv+PPypYD#|<>p=?=7U5wA|=U+A0`_|O}Es#4c@lT-tM|Q@&#{8hh`*?s`6hVA+@#EHGAg>8aNx@qB3rw-(#VHMLlSs#u}A5YG%y2eQ1W81_nDn2EIZA zZ}-fLmj?E1LulY%7?;e)4dnOMD|MXZ-KiDUj;;wxXBWDtE$GNZS1LY`BcUTv>_ei< z=m5@V!P!&X6JCQ|jlf!9KWt!nM~WKOrm`n|ja)UCd6Pa{na`@5MYk2=oXOg0o4|b; z{DS20zIARO8A&?0 zqW1~=1Gxh$F}`myD670}oCY?)cV(_2w-=7rb64BNog@A16SapSa9A_qRx9BNr$71%2Y-qI~{TCR=^G|2m zqk)Os{XR{6b!c(cS#ah|j?#JZ@t<8R^Of)olo?K$KOwWKF@)Km|Lw95OFQe)-8a)t z$%#e#L|^rvzqr5|JPVpL4chY~XwYrYqD*M$thiaP zz?AcL)~6BhTi!gx=g{@y$`E_YB+h^2v5;pEc?1T2tbr2n!jpeKj;J$2W2*DZP9BCmJvTJ34)D0sMwcD(P!-^{aoLr)nZwA96#fERPm zF@6z8b{w=!@g_A5%@Kc1KJiHZQRa&Rm}X&gojKYQyEne)7o`Gz{sp11y>imp4<@q!6oL_J|FrNYJr!xmx%*8a;+mEoj z@zv!qjbhtVGSs6}H+ppHz7E%weQ>rOoSg)IQNQ5hC~!fSQSiK(_M|TlGw0E~zf|11 zbjq0*)-(!_P=f7c-~H$N@(8oVev|S4BRs-b@VqCF@H)?@1TPuSrFeuntN|~N@VqMJ z`{xn1n7VlcnSbcXH+Y0>@M<@*122z|&75a=c!X;fcJKAVi+lI_a?-K%;UfRQzV}`H zLkql%@DGta>MRDo|6ZCofqMGV%vH>Pe>Ah6F?wm{9P$Xfy*43d?o0zOQ^Cu3YAPai z-yPrB@0x*0fB8lBDf86dx8K9Z$+zDzKP{|-o^=IO-DEkJSc)4-zen1K^bXQHNvov)K>8ih2GaGU8%Qg)ajdD_C5EJt zIzA}&+nm@FLwB+VOhPWJai*Efzlnd5(+=Q2jekG>t^E7)bDxLAx39D*c5l+2`1X~y z#3n`BiKm@B+L5tE!^fN|#ZRsLXixqJGq18Io?!j&0N#y?a((Lgu8*R>l(^rV)jAh&)3=Y`6v#xtN51ueVB)qP|tbFOFhl}OSzx$|807-(94BxhRXfI_DSH0 zN%Uu|rxbYLIPR(Y;L|P}AM?^@}fg=4=Q2jGIQd<6S*TAAfO$(mC!VclEpY`76Zs3DNb8nhT#c zCS|;gMdaPGM_x(~@%$=uhK5Kz4w>ut;@9PTrLkX$eBcyqdh0xSsB5)#fHF?{G@H3t zzDnq?yS-A<*~D`{O&RH@>^}lS;pL=n<=DJjPd~l7k!I>bC*kQ|sNO%ZV|#>pqz(ms zS#V6dS8;^0uZFWPT&n-F9`*Un9p|3k>Ii!@ddcC;ul|0M=e-hXSJ6GDc;AzM9de;G zt2=)Z@4Kce4$6F_%pYsVAI+CJ&7t;}_%CY?$2JoGPvFhJlTYN*0@nkflrhSh#^!Y= z{tD@{%d8~1;4QRhAA74AJKtFB53w6_pg%mkL3QYDW=GnURQBOX*p6Y7BXLFg)5n}b zKO$>99sJUeHHfW619)bIXE9)7B`~`Jm}$USWWjOXIpNy~(jp61NNfDW{!df;+W+;i zXDk)jTL?5U6xtZZ-W?7NiGbE1f15Nwr!~WA7ryCPMd_Bm2~KkU$?9aB*YmrF^o{(u z$MDhFmb)nMWRfKTSD+_E$CV5M-K_!5cz zb1xob3}Y!zOLNl#zIVaHonid{;gLP%ia#)9Jxk9%aUz>Frm^q6ceB#jc)`z6?q?-l zco=uyhdXmG$eDh4oA|J3p54j?~4ojCCPrPvM>7I=i+AP9NpH zC04b6#Qz@{(|?Ez%+r?!__Y1Jd#|8twfG>DvcKb5Upw?MA3jQCOfsJ8&aSOXfm?j^ zc)JCekjsYbsHG%XZCR3RL1v`1l#(t>ZibHHtByR-N`LW3@>DW$Bs$Eq)DClt((yHQ z8Cu>=R$6u?8(ZEJ`YHT6R!R1MijEwT9w};9ik}ExXzRjV~=v z7J6h^|8%m0ex6@&H+XOt_;4q9aR>O}*;kOq_SGK>FP(y%a3{7y;{U$fN1Nb}%v5AI z{tE@}nttvTeD&`{<|FStG98Q1yCmZ$QcmPSW07B+M9(dLVy522U653z^AX-1=8XFs zb{JzPBFj4OHz7Zbcx2dZz$f>rFPpiQx}-niZ?9D!OEODdOG73@U{aej3j=uXk!b=zdh9-F`(8{hoD*?S-GsH!vZ|C~EF z$(>|E2ACw15KRJD1H=|9Vp6dTAP^KS3N5y}C5YQX!4|f*ixo?d^lOyZGBzzKWY<{P zb|#HByNxxpX^l0z8%wHDvToXD*Pz+$1R?~DI@HK~KkuEHWD)_jyWidC_x#H9@Z_Cy z?>&Fs^PczrIcHI6QU9V+k+rjJS^G4yvwgvIWa-f~$D@VF-lN#ig~-(D$kd~0nlkd% zxf7wF{+}20kv}Xlx$l*V1yjHkw0xu3Cig=d*BrgV?}IkDtLsr8G|`u0JQ^)8iIqDZ z^*M=)Qh#KF3P(2RaO4#gj-ICvz04q<=*51wbIlbo#l?31r?X9S=A$m{W1q^=SICtw zAeXe@-UUU1BcD-f%hk3Fn-6^Fo6(%mckYVjf5vk^6JyEcd?$9o)95gA--68(7p3^s zJFj<;OMR-1`4PE^U%lCaZVLb7Yoqc_c-u+Ng0a^KJX_?~ar^h!oY?qJ(5C|MX{#{r zy!uorzR5TEKYucOJK@{1cOpa9`YZkQGZqG!<0fm@rO|rGA0NQB3M?U)965x{^4EIC ze;aML@VxE%$XkED@ytH2wEMKgjLBP##Zzq^PUiP&<)6*p+3$ob{CnGWA%kt4J#6}& zPdynHf9R%r)Ng;X`@--}?!H#&39>e(zH_XDv(JcIq&6HCpM~(}f=T|+2IW6UyBpy* z0lwKt-JvC_w2$(KH>l@dUZVW}$@9<>U#Y|`I6EsB8)=*9T(dIo%auNa9QWmJG<1%A zXCpEh+CPQQV@)4^AWiOlk*;{q34PAGnpg>T>VV_E{l}15&JpR8K7N!w`FLMPGUhKz z_RiX5d@S6M*&nf&f!A5Z8JUug$1g(&Sf{kDyK+V|Re_)8)0<&Qkv zv5-9ThiOCB^2RATXX9eVGhrNkj3d6kARb$`=WTs_-gL|cbp+)M9$V(u7mniT6>ui~ zA|6pQ8(m!PjxOP^#B=)KwGT6|A$T}?>*=z$Q?9Vr40)9Q%5#}t!Fe|GFFfU+&>6lW zG_-r{Jn{eJPLTf~ABlS~qxylmII4G4TueLz`*()&SI9SxFkm3H{2#LE`TS?3#rHo>zxcnt|55ICIbYtG(6~gS{ZwgWbflOtap4#r_0!THanf)_}GimqkADcp{w`W3)zV- z@qy1!<4(8#${A`MSviMu!G)Zm_6U6}nsJUZ)UfXglGLN`b%xr+J&@?;|E>9gIphmw zv3?dNoS}Ak!Wn9kFZjUI(S72Q54_4RYdCt2+A?5e?|;l8?|1ARwToEaS+QKfiRY;0 z%Kpcb3fnKY0sq?ze+`|F5bLZ+iRKH+-bQb1?_-F44u53M!{|VL6ZSsxj=lHMitT;0 z&rr*bo}us?rYelOE0SZz6woR^bfp-n=Rp zh(z{Ui)Q-=7IOdPJCUYq{KecQ`te}>oQ1*Y`_ufJkX0fR1|mi<@N`rzJVjlF^?Clj zCCdN#DbcnP_b`f_LiZW6oNAu3^1+IXmJkT(?H0Y0t z_t7h@imS@c40k+)ey7myq|QCyN93O7ThZh0xxapM6lj zMK_S~M1417`>3vn;o(<|EB|s zV{IRWZ+U<}NX}SXe_+mQ67_w_eYAZioy9S~%Eh*y?G?r^`dlbQVv3<~{yXZ_CTPPV z$DNq_siWBH5(Bl6d2u{QUXRcSXG4n*r%UyDq)zUU|G57s_Q10-oX!wBPSE3Ro$ld_ z&%54!sU!P-I&@~=<&*e!x9H+y`^c;U-#8oZGWxBtPw<OvaavLZi^tdvAaqmtUZYjKh54Yl@g`9L%=@H9*pUV_WfmjY0-5f=Z!nr10yuH zX}YiU3-GdI4%=5N`x#5|E!sX%3!h?{-WV2Ld_8y+o4L=S{A+s8?2~xzGg@8#6!uJE z`E=V>o4qG)t3~$=FlO6sSJ1GgUzf&kY2$f#ZglSYqH}i#&+Iiq{i`02>aSVM$v;wV z`99b_$9Z?YkA05k(}{gW{!hJmX5V@B=C&gLb@EOEO@zd@dnz+-+Yxs1ojw{uR^_<-fLlc+NT=#s)2{xB}m^@=v0!_*y|%Z?N#)S%u`RDQnM1*|Qp1qMn!d zVMuuV!zuo>r^xr__olm#S?hH4KJ?=}+q~ZLU&9&g>#%9Wrz|;T@?HErv7hdHVAl(? z{^kd-4g9wAwflbk)@$N(y!LMAfonKZ)^#S*RK05^ck876n)|_j9WRgmw#NOwmiMob z?|64S9IY!}zLIzL@Xi+A-we!I&A|MX-!}Zrf!q1r2>bwL)xgy{9hkrJ+lHSx@J)XI z2KX_`9tD0>PXgwz{I=m|4*ZzkCxD-$>{;Mvbp|kh<+lw#bD)dgoO$`{A<9kybKfSg z^T2@ocIbcjh^lz)8lBOwng6NWi9Umnn#0^B;aBZv91*NNEc zaa$c(Xxm)RL?TVmyR^BJ8(hns+)-Njm|wmhmA%lj*km?cw0+E$y;bhv*of_HKA#02EBkL-JFUMkh5&Svg%Xx}t zh4q%bR%g(@touY<3E#s;9#Rz_8_hLU;4I(bOk~oyz z|Dd1j|N9)aD{^5Y^_4T%A}iu?NcwNn>o%9Fc_U5L$a^8qVPG8g{*M*u?;js8p^Khx zzjefs^1R^3#^)oGa1&<)LlJkdtOc4Pu5nHM;$XugbA#Xi;ibVRcTWkIEmO^6??|51 zXUcqsZt2CQD7rp)Y=dI&OWxsukC0=g^Y$G3VQO&2!m=X+tJy1`@0~Kh9WCpYEZR1} z`TDbcSC>MkU$}27xkc;L=}&f3$Eroa`;E-Q1J}y;nR^E2qz6}UPsYIK@w@sQ{uMjj zs|UFM1t0LXfqf6<4v@1o>+TIg2niMu&D`()Ox@9jsIKE0xoxw%Z8{wB2YEuP7q z=4|@wzr5zo2gtioPic`~52E}xfAJgS!x@u~p8&_t z7C!WuM+?>Ss!jXJaq&Nk?0Si@w=u3~{}`Dky5=$RR|e*xOQomovHoW9=~|gkMa$%yo5B|Mz1%Opi@b-2IB$MQ ziF|)kc;NHQ74j(bCDsS?=b${aSz>yrd#>gFi=Fsi8~x#7FC?^kVrgVwaK`rF!cTJ-ICxmNo;HVxt!zL)CZ@K5Z;oAjP0+eK^#Jz>9G@Y- zZ^kFec}&M1;(jb$ux*!oN1yNeD1N{Te5Kq0QJUtYokgj$Y44ErZ`+Sp55o@&JHG9q z?SXZIC+8lCg9yJ3LAQd_kMUb_1O*m;ey{kO&!9`z7~e$+s%@ZQ&)+Z=ad<-rr`)n4kmh-#G3ovinsk zc>vPSXYj8|KkYs;PB~*lhdVDwE4zR!Q*tNvaR$mr8b{T)y&I<--Tps+HBWru;ueoz{ z27k%B3oOYhiPK;98tDDz?ZGD;pAO?|xAUWa&mAnXE*D*-)_m3FoBwvc@ANXpC4WQ9 zx$EWSl=;+isET{0-kh;Gm`@Ju^q0wD+fAM;=aKEgm&)DXep@c9+5US=Bl}L@OU^xU zfJ>-H^3~k@wcyzsKW`n#`U@5G;vc<>^F4&0y}d`Bmbi1gk0(}-@1r6PTa}{bzhq<| zmt0wCSNbMx%UYJUZ9R>)H^$l)KY!N0(eE^51Aegi(6Qw1-r76X@2B9oiQj%Y_HDeM z@}2Y334NuX12e7<1|8YqCmmD6`LzGA)bWTT+J0Y=|B8>HL*2!9oK0oTa2}|4bT3dz z6<5_e{$+u3R$LW?=MLwS)`p#h#Euu9df+a*{7uSV z6*~RvXxmnLv|Z70#m91j=i*}u{BrA&vi$8oAy;vBrsiIC+9*roJ~i4YXJ2mVq{u#5 z`=bAfPUNXp;jesF2X}Dp1iFgf zII>5+%{#sc9w_*cv-fV{+p3hjJvsKba*vufmJgHD@MFFgdEJIT)Q;=a&+?1iew5t5 ztI}1?*O`lWdyfdNc$U~s2s!i?{Sd!EHqRv2AzuGJ-v2pk_Gf3xjy!&5)seH|*m@&n z{zmW{fHyq8ZeFm}kr@_QT;OJHNxU_tTL&(_%+{@+jLZuL9GQb>OWAIoF-K4AULBUR z=HB{<@3he3TgVsDDJ0iEK5r{|ekFY3t&C_JE8@RXqQ5!EA^UfX`(GIMVql^9oomT! zLQk@2U-ZIF4(EZia^)Wc|Ebh3fA{F=iRi0&@?#tKf#{itaTd(f92 z(A1~XZ9Y?h54GO$?FEv*o331Y>NUT2?ON~{y4+)#>h#KYBF&HP`fRv*y?S20c?I6& z;yr%*C70HocVB$op(*rxu{FMr?r{_y>9d@t{~6vBADigpg7a^sMfQoj6n-gXiSwSr zybH~U@9zy))Hn1nzA*bec+l4Vl24AaLm4Zb%_peJ;;akg5Vp2W^KF}ILA z_|TVYKZKqEZTXS|5ZQZ&JE7F`G`(Ed_Vf#v4uu36`N>y zjw2YEp-$hAE;GrPbQr(K;p;cuUa~W0ABc`z22Xho``~2Z;r*v|PQ!_$lKGpl4Iaf- zd<44qD(gsS{YBbN?AvK*@Qx?99`V20!Z!U-5)}y0~k2NrXFMs!b z@&h=dN6O3D-xKG{dnr%krqYFCg2L1Tr?SSG%#l=I)~c! z)|C7A*Kyxi=@R9Cd@(YNJ`@nU=$po#`DySH?2SI^JGz^=J9Q;}obMOHx3LE}8)Qvr z#-cm(bM&@Ln5+B?^tN*T4yei-^SOVn9vvh95M`2^&zh9HiV9?8{y(B$!i#rsenh;# z)9CNfv}Hl*^DEh$r9+?p1N!<;)^rN<{7>tm{T?CCl@D#~;+$F%k36?46%bMZjAge1>b>1oXW|Wgv`+++`syJS8(`bdN4jl z*(>AV|IwQx``+BWsPv_4&@qvpM^{5b_+o^HLPwSd-&`)THS72$^hlAdIp`ig#Xjmt z&f4Pzj`F;mm~8bn^xQX^YZ{fU1;oARlgP)<;Xkx-8!0<-AhPPno8%+P zU1XDambjij#r(S{djon1tzHrQBV*0ixyM&9?rG>wpFwB1iI`?|Ow&fk^ayR)dj#V8 zLdDpaE@ez2H!^KKK=^+Y7vJHV#5OE+Vc~P4Gl-8Qv5otAw+FfTd*o(6I;D(r;Tm;n zI{ngFk$rzlpG98?{cB2a_z#xNFaG$5GdS{6YEahHxA8$G>J1_0-x505=?r4mMg64W zljGfw(_i@3kZQZwKOLC!FZ^$0NA`V#Hli|3#tmRJm2Xa?yBkFZX$TisnA(dRS+ws*${cw>kffy}}I})uKb8FLNJr#cvh}KiYVQ&>VZ{W?o&o z@!++=JLmHIBlrVBaQzGsprt`)d(@3;8_^A0d|WxY#bDJNxAM67eDC*gWwq z#vy$PFNTKEbF7^m2b?M1J*DWMaxOIX*BY^>2dYOK_NFRhJyna3M1bl4-U(m(lg0FS) zHm~uW{vB<;UUg>QE%q35?J6Wsi4Q%-e}4dHxapKF!}b{C&h5 z9C!}+Uc>*ZQ{P%JJZD<)_$%D|eOMnFyey?u_WqwV)S*8tPbtk=V)`SI3xc8-OU(D! z4;_9V-z;D)MN+ibI1ff5`Q)TnrGv|@(#VZwODIF19+;|6L*rf6`fPuQeV0|_nW|4d zW#{Q$r^#KvYVgW^o@?|p8lKt=-Uk$=(vxf zX@I0V14?oHs$6sVDPolFva+g|z-~Vtm<<*ow zYNYPD-ALcVJAT3YAa&jUHPwv&|M}bb=FxO~;7(uZOW?Q|9|G-fE0=zVkGIgaH*Y*P zV|nR1@d3|dpD52OE=Q-tUR=jJ>({c6P5i)jvX>Ko?2qtm%h;cju{+Wazl6>B>Sf>B zAIC$wrLK7Oishvzu3TO!ZM8A)U-YYaCtrt#wQJ9y^V$91{KcdL;@gkl!z}0ggh-~^ zR*!Bg^sx(^$=C;%mz7p)b)(R*Pu`tHUHFJ2mw->sM~PfYJ_3I~e1qgfMB^maFz454 z*Wt~n%75}Qa`qGAA2P1cm(-f;WATp;Z8gj;z`nsgm=(2g{7=T>AKRFJc-0FhURS59 z?@mAP>PhlJwd+SOZ)4uU)rpKr*FNjKV9H}3dUiiC<~DJ)}lrD zoO-|-&lN|$?y1D8|;rf-ax-z zVx9`v1NsuU(^FX!yHvCE6&&0cWbKjPvQ2E4h4i6nlUg%}@op~c*#AX%-!n5x=5tTs zWMV#~gGV0!F#E>^*TbyoKOnE4B~B)|r0m8O{GS6ZIUCY{5*~F4xLg;j8$4|btck~x ze!X9Cd0lqNe6h926&M#wx{yBGZLGFRpQVk*mZ)ao|4!;Jaw-{~(#|vX)|7ZF#3||(WAE!IH zH(@j7p?yp4WS{u9myzcyZ;s~5ZH6DLfbWz|hBs&ryunw>-qd+Da`q-PA@V@zj^$Kj)^c~@m`Sk0=#F_ z=;SDkzB)slcD`4g5$Z(e9-w{c$2-_PCmmA{4+5WZOyaCX&Wl8+ICUfXF=G*ZsM45x z7p<6X&&!ce_S;?LEuRO(TJ&F&cs29^(6zSF2`Z zlO4x;Na9#aY*}^aFt*||*oqxzz&kwrCBt=G+C;9dvDYyAO_1-8Wu#tv4g9>}L%-ht z5^MMvIW5O{Chrtq?i8lf(UcVw(p z^G+a3yO6(=QlH!Z+*26`CdF_(;BX!m{5OK<$G~%3?~BsTq=r{B)al!)GlTw~pxu+m zw@z?=LU1GUjhOFANx&58Ce9Jg)@l5a`GEV6BX6pF@H0Jz-Y8HDg zI{wsmj~^Kw^(l7l|D(=rI3YO)j9+-oG5V0t*hT01EVOnK9eD+H23AJTv=JYt_^mGi zr#lm5PZ_-QF`GXg-ru9W4UbZ%_$lLZ?Rx4Oc+z(|^aOFUG=;A(7USAwNQ|qiu$;Yv zAC2sj@q~VkABs7h1&^B}>lxcjboY)z*MUKFY8U6C7K6`Az^BKlPCvsQT<3cH$#?Vr zdA&W~uUqPOzi7wBvNt(``R|xP?rA^W!8k(b`SEotc9`h#qPGhTian4D4nyb&MeIc$ zz!t(c*&JGXZSY6fV#k=1<3CEmKZkwv5qmEkJ{a~{VY94=`ItP|%tcAgJq|l|lZjmy zja$9Wy7X}$#M{_b~7@?Gb%A0^eeui2*Qg{NI6Z z>rLoR?9s&XUY^}@!SG6A%|I5kbiXi6$(2c}G>xqRXe%H!(a)yfFIlvr?pI!WF(%$1d z6F*H6^J9H98q+M8q5Lz5;mG)*O=6lkc1-h%n|1IqczWb%=XqnAN10Rcjeeh;28%YH zd}Ka(SlQvYP9nH`Io~I;VNK}sQQvkwyzLd%gp4WlB(W3xX|I4Au?71nKYFq6bj}Tt zeNSxI9{eeBe~a9-2s9(UScyA6Jz+lY=eNDq>kIt#8;s|_eQ9K$@M-dmUMOK)5}P{t zPrhKHkK%0p2=(>In%=h~*>+v3(5|b3x&&w9V|kqT_Y=3O;7>VcCEH5GQW~lyx7jd zryR(gH0l!k$k=~H{)d#mGChwYz9^63(ua;CxJ<;I;QN_Xk$pdl;cuN|nte7`VE3oU zowAM>5;G7yiJz{F@pVHl&GdO6agT39YcgjtH_~7FeyR_CjS` z?*d1@N~|OM_zL)RoDSvtD~LmH;QKi3WeYvSqp)Yhf7i6QtCPQJb``uW#%G)e!Y=H9Z!xL0RUCUjCHGAPC~rSI{+ev7`!x;DVIji2AI2)@2n z*2C1}_rWWKhY2723TKr|d|7zeE69_3`0h<;syylwZdzP0Jr-BVY;Z$UU!%VG`(NeV zEWSG%U;h%%&c)X^%CocZ^;@O+do8{TfJnQDnhIQll`Wl|qgSUpMHrP?{WrDYV*9VN;UfO_A&({IVo2^; zMr2&9(?_;ksHwb4as)Qr751P5&EGk7IT>t#2+IumnJ(8 zT6jEXBI2WLvYv|iDEqENW+tnebmpv{a#i5<4>5txDVB$2_7mMM%e$SbG7xP?rNJ0ENbdi6zBu7DFJ=qN-0@Kf) zB=i#IvIu=`;#?l39*Ng%q2JP$@bc*Wb=h_dNcwKY>gQg`*t1;wO!yc0R_I;66JC5I zB{KS3+WlFqU0K@`+kINvrN94R_jhW;%2@kv!2fdSL*ZLrKhnqgTntZa;ax|p9eMsQ z)E7w3v~#Zd;CZr^e#|?~viFm*l*a06hS%>RUedQDTF(!pO`c!M^YiSBfv>0elfO&+ zbnP!V%R>{p=gxrLScBACqI*x3;(MLkac_9pI85Zb+* z`tgCBwx6A4FG2@vIrS;@5&p*aB2>Un%HTcc-HSl(z|k4ey$FBv8O{TwExV2SlEnQE zrLfsxr&sG^l;eZ)_qH3}7NAvY$FX%GHwUjevt`5e}XIRMo zLUM*q9TmM=c+&dfD4u>e;SRogqj&J-Z4f>oy0Y?L!Wz7a@1wjdhTjR_+4xn({yxU@ z2>304U&(&J_?>b`8C&2Cd&h2Y*mp?=ANjcK@oBOi9_z=zPWIPsV2=-Wz0~8Ed0HZM zMDIVf-(Pge+51f%jNtObtz-R;z;8G+YaFfEb?l1O@$S5+roy&MY#RA1+K%rx!~baa z-^Cj`Rip zRw+Dgv~Kx+8+R!RKN&%WNUm7o{U*})$WO-iS?ELhEd9F)8k2dGdgsOZEc;TW-%|Hy zs9VM*YgK$UixT?1nrGt6dzd*cEWgSxvP#|)ojYD8&*iMH#QRfZz9-^&82jIra|yV9 zenP+3)9*KaAMLl~;>rDhSA*{->HB=U?=lBj4Z?Th_}-XIyqZ3+ZVu^$zE8w&!8vd~ z@!neLW0jnDqd#`mJ>MR1qGEf#Q-Le!m)KqKHnx47d(ZbI=0WBozUO+I4#x7RwmN^NcQwe8E}F^SSOD zJ-U93XS{}fXCSZQYv1O_sx*-wqqAc#;!}O+uBQUB_Tul}Ut`}<)6gM&ZIP3+a4R0P z^sI2aj<>!YLGFvLOl$&ql3&32LGX=uJ&AiG{nXW$!Ty8GxO+De>6-QTbNzuI+B~z% z(rM9sk(_@g{L&sg~K2JOw@Tm*@&?~0zMdF&B6w<&t|!N4Or$olOnY@2HDzbs0ao7dr4 z8^RGK-+-5ho+flF_Z`FEu5fZbK0fPH zE8!uF$bS^m>y zJmItk*#nUauQ}KA0&G5McQ!tF;?k#{#%C;jSVTTSV*86|TiWGpuhDlmP`BL=zLUF$ zWe#6wj%0uQz$54i& z4~iZw@+yh@Lkh{l8UmWeSqyyr!ngwrPH$ zR(e(VES_cTJ0~>5Gl1{T1P?DPk~k{`yho>>gyc&6{Q@{|(?!aC#+uIu?`V z<8+Fn=RfD{x8-B};TSf{AK!_tr9Xb&5e%<0(X+;{DQQD7PhqA6%m(N-7;!ndCve=I(ZB(7u@nDGDmny@1f55OBxVz*ZhIyBLti=92 z4V%u$Jtv;yGB36dn{xK*lTSr^v1bnx2lE*2D+F)Lz@ONx!?dx6bDgSW{`H{}*7>$R z=D&jZMkjqR5B%11FM|Di*OMh7$Ko99*?EGCJ6QYAGw-44zZ-WBrIde$JvP7QSv|h8 z*M=hdo|E@vJjmoI58Y(b7rerzBYE$|Il`l(_iqUeh1Xrp`89IxXOx!ep&wO{=l=ux zpPZX2{g-)puX@|4SH|1~t$h_aAae9CpqZk<9S=I4#09V+k1-F|YT^w^oFPY<_zUjC zmUI$N32}y7kq(9P7m*81e?>kZ+dZP^S>RaqWm_Y1cB4LC0KYkkAEYR}^TEPM_WbCx zp#5wSF>Co&_|SjeDeL9Q;5*O|vSqKeE6Hzhh8?j+p;KceR&_Y?K=KV^SDIDr35 z=I#va9c0rcWR&pBiTm?k*$}<&Ui3_K-8-++E{w(}dhtU`}}33__G zovEy&I9!+jzYJYf$hkCwdG?vWv*|;k?I`U8(1FC~Ho#m5Iad-rf^`r-dn9%?(~{`< z5Tf&lJtkv`JT2z}M*TD7HwH!DN2j_|?90c$03Y8yW}_;TbDB7xNpVMDANL?Ket%%k z$BFgimpuIK{F3?&>iGcQ2p(>fx)b!w`V`KrpnY2xn_yQ8eM*10k8#{tD(CygrbAzm zZHvNNZ+!8Cep?^?wvccCdBV4&>q*aw>XzVD&fFM}(|4cM`KZ+MwBRdxccDH1;Bnmi z3(lhFcOd6xo%Q^V`22j2`r`Cp&wt;A{--#P=!uEtM5T98-X`tuKIb`)U5ndh`7iHz zk9T~t{l)e>4#ttrSFg^9NfU9&6Q|H@PyT*2=UdPV>_ z9g@82-7%dUyoD}Kw$CsD2VJw)laqXLN-(qpS$eS*t@Ekb{!Pq57W}_n=<8nnXuZ%^ zHuN>Io~M-`Inp%isf=hnj(9s#FZus=z0R0!KF5C_^~xD$!3gK3kk21*To|s0-j6zx z8^|@X_5JwyXEImhy(IKi<{W#%URRv0P2IAtY&`5Lj?J5a-tPzozg+HCT$N01T z&Cw5^l{eeALcvFDI=&!bUO$(>XZ|_?u1bIt`OkL~%4a+$b3C?gg`PJ?@GYI1H|kp& zN6SBimM8d@{wVg)+3I-s{nvpGHx3V@>uT5hv*JSdK<~M};SSJq@s8)t^^LyboZtNF zT;Fj1(z)6p2KMgt9i_pu^Y>kx$~%V)~8GD zwQQdc%bKvqE;bjoaeU0TQD+nTY~1Lz%E*7whS4ELVQk$|m^}icFgDvLj6OXI=O(}x zCcx7Z;14ChvlHMCC%_*~fafQ`A4`DQmonP!;skhE0?a;=(RbKyG78@~3U|3bl>oCp zB=%01`?Cq~tqJhw65uZ+z+X&&|0)6gY6ASV1o-O-@I49e|4D%FPk_If06&-je>VZ% zp8)@D0{s01_y-AaLjwF*0{jmN@J|!qpC!P*On`rt0RK}0{Obhx*#!871o*`S_;(5L zzbC*iC&2%i03S<$Pb9#vCBSbcz<*4D&m_RcqOo~TPJmMr;7JMalms{{0nSZ;FHC@^ zC%_*{fM+MbA5MTjngGvFfIpT1e>?$RoB%J2!hesBaOumJ`-hf48Qco&PL7Uul~P_g z8(8sAV98CoAO^n`P3{D1K0USASiJnz$SiQs)99A8kFNal=q_~Eo zKS{NjR=26Dl5QGK(tSxBVMJy={~`wp^#zsMI%ud)Z9&1^x{~G7V^|?W_ZeI{8Gr9m zJxA?OYKN=UQ1#l)tF1Z@xJR#1s?o4o4BcUv-G=TQ`%3C>GxQ;2u30lp&FiJ+y@tCw zPYoI_H1vSO)iF(VI$Yd9U!P?4O;gQD*6=jdo#f@6s^poi)70J+YwtAGmFnU~+2J%- zf37;{av#i9t&^fEm3Ts7=ATXR+AWV0n#HBR={ z=c?){?rxq=*`id{)Ut4{s`7XOyy&s|r>VY7t0zYdX1dxgP~ps5l!AxhJg&?ioh8BJB7Ie{ z*=4A?pu1ALDjX`HU9grmod+D!R)?X6^lkLou<9Lp$goBXUFqP|FkR~yH;P3$#Z=YWs)4|?)N#=8>@`)L z!&_~tR)(xk-rwaN>@pLlTDV`48y@ET{k?Ly9QLSli z=DgA6YRge0E*JO99h~Iu;HI$*s}4A$fC+7Nn}c4}<_2AX$y=E13Xc~ms`Ho~UUkr8 zwR%;P$86$?T@R!a^0;bqRoLSi$x&6A0@r2=akOQ+nsZc7rmH?jg);NHd7339jajZ? zuWHSb^6o5GzgP8VxktPzoRwDvNoBi$1KILUOSU}i%64^mRbO^q4+kuI^SZsN-YX@I z-U>$5nCq%uqg!$z=bCBmz%*TTp{sI+s<|-0!`1@N$aK|TVD6ovh6_AZGt@|-J50%R zckK+-Ji`@Urb07Z2bZbNnRADhtHz70)@7<{maA&HYMbThUZ$#Nn?1|a-q~Hqg}zJj z+Lo!nT(g>|b6tJQRPWrm?aNR?JoU>|>qlK9Y;e2GtmB<|R`oJfIp0;WTvc3Q)h$=O zSGanYs_qhZKW~;6G}6k)++EavrKg6kuJrB&USNfms=)#Jm)e8P(duKotA>qgaa!-BpW zRm+W@9<~gvF5}IS)ev><8h7hj6<7l~*M73BeXVNyl&58_s$J{qqyp}^BthSqU z?@eY$x$3(qBqV#YtG!%R++wwrtIE#`p&$IL5PJV-&DL_&wa#kc&2_HEay7E9xT#zX z-RkPzteQ4hU7JDRb5{Rm)wXeN?`GAw$?DjwhQ8npZB_?2Ta?5X zR^-4%x}d~8qE*={da122N+n#xP`#Rhs0@g`{Ql+2pZUt4`O2U9%KyLlN|m88q*~#W zt*8Y~6-IPBRjuKwbgD*&YsgYHrd8usC=&;rYS^5~*89E5-X9YT^M9_vIp{U8tx1UJLcKKG$7|%A0GULG(;R1C6OLf9A7) z*5CfDzx`Q%`|sD^Dh+)`*K1d!6P3+X2m3UH(N;LDVea*CSTNKU(*^(RHD|(0tCPL$ zlq7pwovJH&JNzsq5B@upl2--Kwel*Qs?BDuU6y>*XTyWSVqvZwsk+-~4LWtNu-IYe zMhf~oP`*fRJ_ERz0GdNYXPO&)5xOa-h zR`$lJ(#f`|0#{|aaSGIC!eT44Rvw(DnzG$>(^OBkcnKQ4o{DLzF2`!bq|bHXLm0}n zMsn5OJgY7*ZY6$D9-Hg7HmaG{N>%JrYKv8=^&M7~);ru|W-WH@EIq?*wP;?vLoKkd zkgo=5t=j6=x?Nj6T6gM%4@*-9v)xdYxEV|& zwK$z>9c{WMN%xpVWWOa@dy{lkQo?ZQd#S$MZDN%cnO&L@_iN>|nzh#jn*oR>@eerOlZhLHAthSH2=Nic`L31wUO-TZGBo&8~M)jOhJ+H*%3Y0A++&A8; z%^?RAFq48tV{WfQ4I3uqcnK+QcL?0;D6Tfg)^Mr5EYE6p=mlnnLv1Z-wkrYb&?Pvw ztBf+>kYNE=JG{Wnj=XM1{BxTg|98Jv>fPvLoo)$b_j03M&s?<=K=I!;Wx-=o}E|(WySCzZR;PJ?ml2$xzQ(eFZJqsFf zmwDWn)J>Ts+}1suDdm+}Qbk?X0^lJVuFW?4aN!{~aK2@monAGP?ZJCj@2xl5bK_W$ zwYW*&R)E!^eP+O^?kHjRW1AKb8pBe70|2)eodzQVAu26e(#<5jdRU z0U_IE&j!d4=)Xyw|4bA)#sZ3T!RuvuPTzSCy zeA?nJ8O&1sljWVZDK4&j>zv}|oslWhZuis$Rosi|v3OeNv3Nh=@$i1T#{=B!aZx^G z!&RAeMr(Eq8Ic2*>yPAl+J*OaYqbhqskJRwZZkYRhK9*iICPx@7N+YR#iKPvWqn48xkH3+o7S~D;bByF zFlQ|0lDTjS*$Bj238@hpx9nfLGngxxA{11lYaV;)~FKjNR2cFuRR7l zXr-c;5+P_byl4XLf?S3>WY9@3v4NJvn8HnzfC{#|2q;u(ccYCe2HP%m?KQk% zLp2#b@HJ%2gkL%Gn#WjiRIV(L_(rTh&13x`$k8Q1rh_P}@NmOBLa<8uGiX?0LkEPl z>$b%Htk6q}T-^?Ji-!-uUdR9i#6uVzJ*+h3S&hLk=#i_26=uLe(SoZbR#L{fQ+;E# z$voKgA&_qI)UpJTQ&N+~)OJaI)rN~_b%sco7QUW4!`tR|rj(WlESG_@TrgjSU(ur0QlPfTvKC83{-bm1))41<^(JgiPr z2lN=$8hv+I>s}2p$LEYbZ)F}updPd}()c6lLHB2=Vpr4-kF@bB-L9O&H zLU*dz5mA*sqC3U?b{n!OV;*3gdzJK^*r-~`TJm+``qMjj)35-Wj1ePXsm3Jg>_~C} zb|=+QJ4SL=hk+hkb4gVcm8F(frrCW}{bZ?e?X zn6eePj>%v~omQ6D958OM-HslHy2p}6LaXGsWw{mrrK_`nJi3itEu|6tJ4&BYP%@}OnIH*ro7Q8 z0PZ$s0*~0`m5xRkt!cKHx{Hhf6Fbg5+6;ZN+oAR{t4`Xfm39aR(@wM94lU3Q_0bOH zv_m=V0MiaI?a2EM!BxOCn@mk?wpUsk*UtNqU92b8hs3b4o+Jtl<7qJ#;ASx3%6cS` zW()PSys}C|R8wd!pv!IQs5SK7vA2YV(zR7myhU#@?-@ z?gaTUq_~Q0dbx=GP*d-<1sn}HX$L3Lg$~Wl>F9C~5tF7fy4RrXY@(l4SRFJt; zk>sgm19nn@J+ijAv2E8aJYE{S<2*e668)ju%pNptQMWk?aZn@F5Lmt13TchPU14Yz z`Y2}YvE!>MR+8)>GQ#UdgHWr??P|>qVe;mn@c+EgFaE!&*W32Q0_9=4SD6!ZHvDBJ z;#c#TZNjuskkI+jA!@bRD5|1(&1wu2PtQRE9ake%fZGi`a99r}N^Fgta8v6TT@W!^ zIJf@)UOL-FI^S&~qdc-U^0uDWPiBwp9Ja*e`cid^Qn%D<(tCzAV3*;>71L*2jY{sA z35a&fvk`~4&eS#LC}~6vppJfx7m?_ONuJM$QPh@#+3IOcQA1>?rI4FuQqpXib)alj?DSDGdzNIx=-+Q=A6#se-{M=U1Z zZ%neL(VcAeChKsrmsT57%;pr`VfUab#p+AZJ>&Z!xGYzy zyjE9=v;>#khI`n7VFgvyJMetr9dXGshqu<$dlPUO(bG}t(0gn2+XC+o~#5kMR`vT`FRfJ)xtU>^cr8Z z@aMF8ho?t;Xl2oMCfQ`+>ct@ng-IFu@U#VDs#jwoi3YpHZW>NsW%%MX*?n}wsL?jG ziy^gQT!@Z$wzA2nkG2YlVc2cEKBb*Ref?eVDm4?lp7Wjet+O+hHR5I_jjTmX0H_@n zh*;@QU}e!UuM}6m*wFI!T-)Q1%A06z^WOWBxsqeKh8>x%K2w!fL9Zdh+iRjNd4^2r z(xkZG^j0S6kul1M;f@}FaCfUl(Sewxy=;CDn*8o^K(eM)k)*@pzC+fagKkHMV~99( z6PSh_cwIg~m)zf9|X4ruQdL(5I9OK!vd0I%r!iJKz!uOJAQc{^x0@+4!GDwmysf)r} zQVOEu^SEN`HKHde=Ac-}Ze9ls$(Wa-#Y9tQ7(-;u`XF;u>gbUs) zefobeb{VJ7EYoG?s{e*@VmpIJv4~lR20Ti}1i5xNiaucf$Ky9ON~c0=QV;wx#;j-J z?HtlG@$nhtmE)HkBSGGSFO86C?qFt{`4m55r$e_V5|lm9=NrH95%{g*zk5jhtNjdc z*kM(gdL*GD=$$leUJGf74-FE1wcCT1+wHN-bJ67cbwQi-M{v{Ra3SPcqqupWw&Qrb zyqne}E_;i`g1${U-EaGR2FFxriHm*|HqFAsQ;hJ%)ypJ_a}!S@tOLs#&t#?asnOxa z`AV+!D0hp_A$Cxc*g+`X#Z`RWtgRt&ELrH@{Ru(mXk5p`Vx0+lIva0)FI!dOp)OWQ z3Fd&>VzPX;Zj8r6s}1*r3AfwxnbqfxhrU;P@NQgjT9qGUSe1r_8)Vd`lUS=4W;vI! zVBb#&0Ne56fOhwZj^SeRyKGk!el4tj)&uT5b|Z`-5_-ujw@}zOB1Q2ki0&Q`;uZRi z=q%q#w2*2o583Gej231;J8-++h*uAfxt`aFf@@ffD6B>vx@y>PH6Z|E&Xt*aEzC)G zn0Jz9_NS=cq%G8yZ1PcOGFWU*akZ!Dt`rxlV<^QMPSGPNu4+rySk^&H*NwTV?Ddwn zvZP)-uppO(?E*H||h&n0-0AtwHdm%m$k&>lke zN{Bk&}k zit%1UPbftJ?HPS3s!t>Nsi36IL@Wv^)Y%(rFkZS$-Dx5c;%+?JCT@VzXe*8wNd3Bu z1#J7=hFB;j61AMZJKWVK=1>8XXu43%1Evcf999Swc8u58$d!&MARgLpv`H~!h>{mJ zsJqSK?!m)m&TKTXWvo_Hx0`Io(7k5CyL3i)&Sf#40|Q2rlbA?_u?iS>9qckJyFvOK z)(Cq^93~;g;RLrmV`E)x}wh|q+ z%8oSPt083hZZENXm!qc?n762zH<6!#pPj8q2~%dT$#U6_RHyS;s#Sz_ga~elwkfpH z#9TEQ1r+y8+|XqAOWu(=Y}Eyj(YqM4{SN*Z5;)qmM0|R*CgD9k#WJo-VtQ~%s{>1` zNZ5u2`8FCARNLsVS|l=Mbvbm$7~zVnyg+t$t#X5ot<UqA=Q8o6hlItRq0>P;6Vjk4z2O)2g(3#yWI#h52g#t1GJh%0NA2$441f@2PG zV)e>&#TTpiw{P*9^$t~F);eGUOk%M`_;a3BLCgym4H2X^Lww$(K9T$brP*7qpbFHh zE)9voR*18N?3i%CF*9K5en-hcQ`aRpxb1aQY~G~ivH-SNJMUw>BqM2j1Zi=ZwFf4+pX~0Mq7J6G`={X1cDGI;U)~cSi^J*en#@wQS|+Q;G*8uJtV0xIRgvMUnWFZR zs!2(P**cZImT=j6x2ro-HMw1V9yY_dYCLQ)cahE6=k|o%s(SL84zF&UVzo@sZBwky zDY|=#)i*_lrdT6Wbp2E}PdlfYLsNC7$J*=BwH~Y2qw76$Dew1KU732=a}#h+rgbn& z_h+sG?#(i*vvpmz*`BStvbR#w<{h09aElI6K9luYMXsDIN{bj;kF7(%Jw{|Zq`S_6 zuQl--d@d!DyRcHyGj5ZefHm67ca4&AP+_yemJ3QoHxvD)XkSZmOsgPu4+DeMM(nk2QGu_qhQT^!_CE;{; zlSfresulj0aT8M(&VWA;yGuxWZJT^G1(j3Wyx2G;kGeur3PK)LJ=N9eQFT-E=uGQW zSCvO~PIUqIPIZw)8=h)adUTD))9BI79&5y-yFFH)M-STYuxAt*kpp+?uap;$B+ClJ zgV7=wAcTC72&duhO-3=Wx-7}Z!-;|4w1vHyI3n0*VY(=9Gn?r2P?8&k_h7Os%;|o~ zD6-<~p~}8wYtYjDDFyYG4p;@9mhQFek};)Lash5wY4s68)H7>nLQ9OMokme=9CLAm z^^j7Lj9Buvavy`~YQzXPy+k`&Ob<_+l2*3j5>7(8G$vbsv>#u3wvDyMM7yalyj>2} zhR@1@j&7mpbQ|-KM~*y-4?3=9CLL~+p6~=43%k{gnmd=Q(-OgpE@~a0+6n!=PkZsK zu}0`HPhto=#I7O}P_&XFeq-|V8@gKfT#GRiuy;Hi-mW)o^mO4B*RG*t24Dgs?6Eyi zIq(S)%&ljll4QC9Ni&(8!6b{Fv%^U&;V!n)TFi|LljLAZL8lXg!{R_-vebZEEUUw* zyR9w2ZBF+XF0a!I7ZlVpEAW2@nTV)4DebH|RvvZ;yMv8#iN$*GoU}WvL5J>iSY*Le zn|CD2XMH#H68mS{iF3&^RBKQ%4q~WijF!>qv;BZG$t*AC={b2He6Ywuw62WG{k$S_ zdlBN2FN`|20F>V0n=om%PH*e|^O)QPWLcj#W!r9hEEoGCeIdJj#TIVX=m+Rhw`~_~ z-whK^Fz^|p^m**NOV~$=f}G^rDo${?luttNFD1TMlDAjkz(Iel#+piGOx#|uP zO~WRRL3eu+>v7dV32wqN1P78AL5H?W$iE6dB07bOg+kZb@G2WMSOypkGh47t92;Ro zrqyHWkZBH^dT`A0u=VgFkUtkWxE*|IeBE(ku-kbLd7dbr5@BSt=RJG_o-(l;R^D`0 zny~d;?0QsZQsLqByoOcRZs^wWYef7-b-Y(+`{=IC%!nzZX`6W#lu zBzK|M=uVB4iYoi?1!u8&hnfq6lYeLVJ;6`AO@Cpl90fvzx>L?^u-dVw(B@5>>mo&o zSrcHyP_R#x7;@mtm24m6P=zTTCgf%>4iqB5HfzWKfu$3|-H@!SlV-Lh>yD(7mSkO# zJlaW{=C9HvZrO1%Q#@=7M9-?!GW>uRsn@9`Hv|QP^nPmJON7#~vc}ZiB9P;Djo@Og zcn_`?rrT&cXqppnAU6q2G3a4kkPYR|TH!YF->ytatpTh})Sk1UMUC4odnW zJ31tDvCi-%w7pGlMk_T*wjFV(tIZCR*a2L0l?AqLlTZn$#0da%X~utSqpS<$ieFhW zMKKOXNL6Bw6;3Q0Wl;}Jw`sANqR;eH2>MDYlXZL2*pO|VBG-d8fsE-F8ABiOvf8~{ zFxore_4p~O9TIEna9BMKVt%76IvUfdgg`2FF*a@qFt8{PrQ>79!#i}}nY|BX~r zJ_$I`8qnt|O;ivc4zbQL;U;5Elu5v2a{B%8g7eETc$OQ7Gc*QGvMJ1_d*&0m2YHR+ z&8Z_!;?M`lG-Te4m5t)&^u#eHbNfN~g~)t{yQ%=a7-0y-wi@1g&}7gOaTbazO;-Ui-Yo@eXt7~I7T}fIq(n6(o`#bmEBtWQLUwr=m=kt%!U(UJbp7-;6|D6ctU9l9l)Mtj)&2(%MwqD*Lgw{xu z1IDT)mWJPj&~igzS1de8D5uw$vM-kHjxx0&z&xr&K%zO?bQG^eC+&%2?J)@hv8+2L z;Xo|wk12t=V^R`|3`|NutXI1^3DH9b%qIL$<^Y+$+2P4a&9OpjEamKuHFw4eCu5U3 z;)K0%rXz7eUt9w62jfDkM*9nTR!c$(^pS9ssg$A6O~4mzl=K|D0eDjwq69NAcB`mx zWmgY5GKDP;x7%gO1qhWX(%)9kmhKyNx;)0G&`&zA@ZC-TdM018^a2^Sk3u?M4#Jde z0)-cL3kc=G9t1Ah$!$G*#FV2V&Q};NIt7jk#|fMUpDM2MT<|44Ze);r$c~W@*N2H1;9pj?Aan#9{08X}T}y`X-Dkp}9ZRSU+WL zFjt5raIkpVi|9LalY7FX1x+s)+1D%__wxDI_YHWKc;Z2liJlh9JBjuJua}xY3LL`%PhoZ{*Q7GHa+(+5tGI!zBt)j zi`?<36t0R%88o4{VaFKbUMA!wn2%xlK%*pR7K-rwh+qXB=moDpj%O5kc0x=5&D|*# zA_p_tDhtrbjM|4Tp}O1zZ(Hnx!=%qKL1WV^HsL$m|2jx}W5#!A@OQ$^4}s?f%&GZ^ z0hlsj0A$LL(2ws16Cy^NjixrEu*;az93|{CTEQzDP562;q~Rf40J$p90(4A{!q^iD zKn8+TQ&AleU^B0JAC1pKiR^9faUD~3lNX|ViaV|A0r*C!^xCNL-> z7bYECj|TD*KF$##u80^jj(yxz59-A^v60!R5UUHhyQrP4&B53B^D-H0=uW^35KN(U z4>LA?`g6p_K`GuV0x6T%X^WU}6i)6UMdA#IX6T8auSBp-tAv0gs5~(Y9YJva+p~B# zodNL{WZ{r?305f0jtKFP2KI=QbC`B3+!OjGu3D!&HqqYHdmM3(y|xsCQU$sF*xz$6Y}r&C-WBHINR*M^8`1?mtW zk>L9^O5MSq0k2!obG;Xa;PXEJ!(B6JPP>?brgdB;jAB_IHxeLge&HGS`w2Cp0MKsSSiH);W$^CM@PMp$s{&45$VS z3^7sgVggfj63C6Bvw*mVqyz}mCnfYB+KGL#&=bPmbjIw$H62im8Vs;O9F?Pv5+VSB z88AfcqPC%PI3Y(3N^NKX!p?E{j39!B*eW9Yy9`vsARjgeM-Apyqj17tg1UniVswX9 z-RfRcf=N6{iX2Oqh%Oh9-vg~}n0A+FzEXx*M=3xC<0c}uAc)Y9K!*dqc{h31-_DR5 z`VfAJXTY`%a1E21g-Q$@F~TJAj4n4KB=-mo*iCkeAQxb;3h@1?k%9+yMq#(_jUsp3 z?kErh$T={1f+=j{C~_Ayi@ZsoA}4`KMa(Frql+kdlS~|xroCBYZbwE@WL_E1lMyFu z1(LJj(3B*&1xJV|a2_FG8$f_SKK(ZGnw@K6%&PWlR1F#y z$UUH&U%vvoQ*fj5XDKi`N2J+Ph1e1(A4}a_j-QB#~U<+f)biP_+lng+g9Q}^JGvk z)uN;T%RvC*+aXAf>D`zBuDOBMZ4=*r=$8fTSOPk&?v)bmmV)m&15$T%O9DGS+0r+i zb;ergd#t4!mq%Pko&|d@gj~Q{r^g@1oe zCnmnx%z9$bKiuJLLjEvaFM+bR;%JfnD}Fc&97QDwI`IQpP?O?x?vmrXvJm@#3v=1N zS(dI0);imAB!eB9ZRyKkCoZ=f$Yfn}EPa`5c#fqD-;?9 zo569C3h<5vd$j?j>&Q_UHzlyokV>n z<7C`oh+8~RANft0lbYrE9Pylh6|@@E@@VX3A?3~ ze$rHPM2K`rPIyjKe0x-2^QEdtmAI6W1Hov~;O zOq5VuQE{3Zq#IZQkr7JM`wBPHl7rQ83d{pV29{P42g;kf;EHa9@Y0R#GVF%uC&L?`qp`K!{+ z-w5{WFhH9QpABeGC}_exrYs?*Bbl*c6Vw@CkEZ`sEp9qAB^9Y>> zwgkMcK`+TP&Be$Al5iA`CNg!RrHPfQ_<0I?!09a5nm(Z;SZ)e56J*3mf)_xu1c_FAe+pP8 z;`B`D^{_xV3m=*XOgs6nkfLO_hSMg2K_4Ld4NwD>nanB~C>l^YIkiz9@$*U%YS}|f zJnR;AKoF7QJI-;r5vZP3oSD!R@G*<)QYS2H?0(}*C3Fq~y`=$$)Wk?O$dX5Nlb$%7tc9)3h z!R;*j$YfT4b4AAu;sEiLe9x4`W6}ErGq%(Kxv`_(=ps^Wl;bRGWx!W3{8@I<0mDWj zpQ|kB2H=(iwHZ(=Vtt^w>?2X3SuQ_8LAFHxPa5HECNxu|T3ZxiY=h^S{99ZM|?ems5**Pl9XW0fS}t1wfEyd_QjJz**cGoz*sl^+n@gBgD;&?}sLx zq%#^94~NDUi}}bDw#%Zx>9NORfzxBZ#j!61}v?>#3cE&8I$*sOB^;0%@_GKyOR#l zQYUqjS|kZ-k(0C)pb+Vza7qdeSCvqQ$2+eRJ-e{TQqDG6(H$;;?-Z~US{G11eA8|5 zJqA{?mv2t;5jrH9;T<$20S!W!F?Ad0)`I~94EDk8O7+Gk5Z}#-ttxoOLz30U2HFwp z08)zNkB|AmZ;2!glSz!wEXc-5I1BNZ9XN*Ry(2r3vBw~vP=!BjKJYF^4w8dyh(mb{ zzX}o*Md?x!eGTXigA&L>@m(UT#HNfS6A~YdA>fa+YZ3WqzLw}IP@@@Z*(%ca7FYym zErVUfJmOxRCNk+m%;+LtLHZ6?V<1&rXGcErWPs799pyn=6Xa6ChB!se-$MtY_nppo zE1GE)EI_2~0-YDW7ci$B;o^7+ZoU&Lyyv znueC(LJ9@z8e=}b1lB^TW#1%oPd@ACPNoMY%Psgmc{lQp#?tp=u{c`;ah6?qxXP^n zlVvgW=dryO+VK0P$j275zA0chSnCYSk;SZi#t zbvVVmH;)}j!3OSGAn#ei4lFS3Uc!zoFdtpQPA~(PZ-P#pzS55o&w>-5>s=& z(3Vfd_vFjn`NH0O)4rubf4;eIsc<;I9J_KU&sb{eE)WhcO~Feo1ql#!3M}~ASzrlY z;}PKk_}&36fnNc#+{xp2IgX8GZVD3s=NX_WBfJ59>oKtzcI<)GtTigW_i9*<;SS3V zn=P$taAhBGCF`>s!mm)U_6xAHS^KnZVNWq@y~x^A!j4>IYA<44Gb}q-vwbsw^m{M1 z43vP0H6x=hF`>1H4J9U?xRSNcH0>;6hi01aqaAItENxe@-LtSs24^LnC}Pc*m=6`P z!-^T%eY4wyy;rhJB=KfWz`EtrRbh)|jO4dBbj2?8(F&(&) zo#5XO&utctU5y=rQ%XN2vbPvNB|`YBC*`n#`n+(TxU+=y+L~zu&AFx{SK)U-4x_+O zo}(3QFWJav>SVnIC#lfz@_0ICqBwYg<=9mCV&k{v8N99A@$b}lT)Nmd%`^yS_UWcQ z3Bs}Id7;WdPefOK?hQZ$UrMkO0KdN(vyJp41KN2w_mlz;j57sBl=Y z0{M25?+~vCFomuP$it*lIqpYE=we8e55qo0szo>>A0rhdb>)a;)w_cCp#U3zYbCfA zwZ4lO&by6NenoU-o5d7x*_~pcS~tq?77oF<1#dZYir#O*)uUwYBb0}T(IL*A5fa&Ca5=00NT`|(^ai>{D%Viy@Z)p}>c z-Uf1k%l<&GC{XUCn2fJSrDP~ncamg6?>-tW$SEe9jqt+kMq_*;xzh4GnJR0*BqV)#XWZyQ2I=gCN)xw22HE9l)f3w}=Szt}gT9|6(@6`J24RvXFA&}PK^KNlD zoWAqc)_HG09Zape(Rtp+pf;h-^td%Shx?$~>vQr2s%Z=~;;YYTcWfI!s&m+4Xy^Ie zuIh6%gKF>n3mOA1H@017YG(RpJF4J3x&lsLTEJ&_1^j9B-5&7rolXY_hj0AUp2-0V zcsJ9zOU+uO9WP&_C*Z2bN>M zoq`k3cT<|*>kFj$oo;9K`NL%%r>nNE%Iiz}T%+?`p@5q`F6`Tk^i*}{~0gIIAP$ne#A2 zmp5`#XM#;9*qx)!-UM@bCe57&@*agjC%`i2a8iRfbRyz)_AKGQJLu1nUY##e83Dr* zf8y7$dPM{H`0>Z{Otw^nmQ)7P?zq`r?etV{bAx?9&uyK5W;KLvCSV=U;hFd!YAfJ# zd1}wmLfQRw_4bAZ*{SIoml%=%Jco1kZ{m@2*m7vKRB@IN+4xyJhYg1KiqDqBhX(pt zQQT0ApQ0%^hqD{pNDTpBWIBHi8w(BavtVFCt(|fK&x8utJ=^NMTOk%&Q`4=g909$N zDt?|l1&>gBO??EKat>D}xDC!9U(#UrZ3f1zaC-w4)EtDcMXBi-TxJM+HXfIp$pSgX zV`Ojj=J9o(!zm2MQ>s{ip+YQ3^VkEfEzUHz{gw#oo--MozaCo~CkXp5eW51CMkF|A zQh6O_uFB`#N;<{#tki{keQ8=*dKK z%{kl>Aq+&a>G<7uE@uxr&F#nO{ol~a|0O%&OqPfA4;9r;w;P8OI;Kp%v(MS#j99y? zzM%rh=Nv8kSbZnSQBbFPHai^^=T5ZwcZF0U5a(%tx{1Y`!hRd{<_A11AP2>I%}Mg`RnXH z$ApZ*dDxDpwzr&9#0Y{Ja5e>W^X_??Mjbust!e4=+}Z^6W+J?DPUcl3G~R}DC`pcG zR`uBo5}R|U_#a0K#iD%g4+p`&&7XFDwG0yuSRd#A<(?1cWYg-4tRGL!fHi)J z&uMF+8Fn~p&YLWvk1f>NDCYCLB7+ERpLe&JI*477$nSYLofC0Q=X95RR-8L;2ZY|^ z^q(V#m{B@L7|qVTj!`$$LQ4v-FsL$v?vZ#Bu#=OB-&KE(bS9_c)iAL~(K+0HA${XH zw|G=L{f#zKIFB)KQtzwN($g0%a^~2ovvRU>s;n8#nnjDUoeLLQZ5dTL)~sslqN+tX z)`d3PX*tE7b(Pg#H+iMGyl~sC3izB(sNlD_YIRKk)t}~YHu%$OUG;8q2FuLQo%_gv z58@qX(xJ8E4!pX2(qQ+vxCqO!{~7i&XlQ|tZyMCyu%4(T|)!EiIp&96G zD<vP=)GwRQJ64Q2l^iUJ%(xqYW`ZW9tXrtBdpo@7WmF(XsF92xM$N-}{j&O-)V1|G^<~UPIkW zu3MbCEIPI}8e{`(w^;S2G%sg+PFVyzMfK(i%!IGT?cJ(NW-d>SH&T~I)js;wPcKgH zwVE>j^a|HELXYvOMb*9e>BThs9cead{CvJ9s)NiPP{8rXnTg8q9iWXa+*A>%IbL?O zzwd;_d|PR+ha7_^EU3xUnnOaG8sIZgZ$}B>4n*Ae8j0QXgN0Ci0LL~HX)lI$CQ3JY z5Kj~lHAH$fw2e6@ljim{M(XNmyn2SOrPUK^KGT;mUVjB@P|p`fz-Pcp1_=uSwcx_! z4AglYs+co_E-?pl-8rdDw| z{EEw?P*{z}u|So;)r6wQ^Z1p3S5daQ+-}9$RPA&+lt7(Rskb*N4reU{HcFxHDyL8J z)`Y#~@+({I+wihisY7H?sLfEt4rc(epVI+09lMj-vHSh#X;>{=U4c5q?ex?FC|m2C zp0Jh`G_iiOtD%8jb~eBOFR!3p6H{P@OW_%H;Z`(=E%^cErX1^iY1 zLaL7(3!Gc%ky;mqtImp257t-lH&)XWtaksWgyS>`?4AkU5oVZ#@E7X;^4WbzG+wsoQ>D zexl+uqqN>5+9$LI|9{>8H?RBuUuQyR|NGCkey&$|JvFY{mE_gz@_3xSHH4x9c3+^> z3&Ifk=3nCtZ1A`cIDE1wAin@`jjC-tyo=;qY8?Y43(vYj%Y~Nys6D~YZih&1f?QV=A;9Lpv z(?Dpj5RvrD)~r$V_;6}s(JPPboA>B@4fraC)n#Hv<(E+~>Z1T7mgs zHK>E-eBIQMlK*ySI@EcyA=`|i@|H+J zz~+pUw&#hLQO^(V8{kKad_oj*3Q@q$Gge%ibEmj)+P6ZpI!EN8h@dVl;j|soG(O2y(f$uv?;S{Me@aW#>Hm3g~cI5e~ zHzvQZv@LzntH&q(W%n-c(5o-Z{^#s_D!$Y7#@8N>Vd+o*>R%&2Efp$%lkffWlE1Ew zcRQ8~3sU}h3C;vxT9L`j#4M~H`#PVvIBdFrix$1Y9;i^ylKQU#P7+=4VM^>Co!s8)HkHgQ(>fj{5#<^8r z-z5Eu2K9ZRXV=hc3QR?cDx$7Jr7PU%1u%{uVf@U!yyTM4-CgzNCHGFvY>MUB2anuN zJFU2ST5}b~aI?v%&)Bs7P+`qO^c^kc=4wIixw%@TZEo(UFn5hZwYzD2q%+iy7v@+; zCECepz}|8y_FBv>3@Vs)@;j#v0`FJDM>uSRIJo*4M|T1#1f#LKGdcxqK=<5KRBa15RoXL#V1mbl78kR*RMZa%@r)0>{1o4b11rr?A%I{gjKdaoZEDG$_XZYaJyb{|x) z&cL=RC}Cy?OR8T&8=P~&FpiC)cmF%i$7T zC62XNYu{O&spYuVz-jjsdKZ7y+gnAu-C08{1^1w!eBhDv~6ul2fsBv)fw z`nK`AJUFZ1tZxWxW8Zk@=?tx@GXkpIN_Yg~l|6_kxQdSyilp(>b5oVFI`GPMTfD+oG@qlV+n^EoT8nS$}1HWf}hq-6d; z%Rn<#PR>U8)mI%pU|(>z(3XQk^>`bhYlV>DPYodtaLk*v3RV10O;)ZXM4Fo$?2Vvqf!{vX%xc=xiG9zXtlevY72V456EtSJVs>tIzSFENBV zjZ&7WGfIQA^Y#hV>O{fi2=07!pj(MR1JTwi3zS+gn7STBn+B)`IAYILVq&x`onom; zO+qdMYwUEaZLnki6%+r*-vR+%1?4eLCr@G+P7fgjU1h>^S7B*&5+BSc;!cY^xs2Ad zc&&~Fh1m?X4b%xdNnI^oyUY#h%M}ttuu8At zkFH(~eB$R{wLY&WKO3GgN>Mz9-a(pmsHMPD0oy?iBr?z6E(6#Q7>oG%IW3x`V+N$y+3+8dObS4P%Ed`tM@~PW8c^F_4n}$75sCN2KD~x z<@B`PUtE98KWnv}#n%VBMpr6c=_8gAdY6DYr`md~#a(QiK`OQ59IfXUr13dl2UCfz zW1|hbywT+bJ^{}1H4@s4AkTv;I4bP*XeR(wLHemnq`LgLCTOeESK+Ox2yFEx&r|z< zzVP4MtubX6{^iD}e*eRR_kTS9-sgU#EZKPPH($$tV&B)D@&RFT?ZStqG@5_#Wckum zzcTRHT%3V3=6ku&?x;|0Yp}A7`=%swI_4+lah|)PisqKed8@14jfAKwNH0Q$En*rg z_zI1BhYIVDp#^TLsCRi>Riq8B0JWi+)cFaW&g*HN57*P`tTXn=tlcsHe5>KRU!Kd- zlRBQiIYeBXTfp-xXMfFxW1_oLp+eveGr3t=*;=98Tz<<_ZZ3aT4`PPx`5AQKvm0w> zdd8y9vb9XxXW3eImb(JdV53uk8nc?)CBQd9DG2==R}XFuKCU8xYpcrv<8Y|pEs&-3 z8b;^pFOJTQfNMy*2pmIRDivIZ2yr>tGtaK8P(NKjgyu~nm2OqxdfKhPC=3nIJUXG= zuMZ<@Ah}yp?#0BcAO;n-gl)MoF)<*5us;L~=f8j1_&SkACinz2$ak!q&PYozrnvV)V^E{pRMv?+f>kCeN!R zkT1mPa7P;dY*l*Z z-a(q+OmZ9a{iO4P?zKU6+R-Ft5`H1#aq*aQF{DDH(NY8W0`^~%$006~*zMkio>Vo~ z&Q`ATHtMP_D2=zlc}DFqy*;k4xw-m4=#WCC2u^iep~_TtevnR>_*MefZOjKW5!4Gf z_9-yQZE-=;#}KNVaA(p@$8_p+{)*{}6b<428~efKfiB-q-a$akq&%n~Cu}l@p$`Ek z^q|#&MyQjP1f}awDKw@!wVK7@)r=`Ix3d!cg<*UP9{9JKA=7Dpgx7@Y}qa+fkl z4e;oK=nJDMcF<|!_fcb^0i5M@jGz0k85z~sn2C!-Mw$Y|vq1Iis&P3T$@9jSAq5qc z!7P&%E>aQNo}W@=wq8c_OLHdA(==bXN(wyWs&XS4Y(4e{^>P+x4X%^}z5{cYuUN-- z(yy09M{kBY0d``nHypN4J*LwrI$Iw|Ajn} zs_ypd6xwl2^yB~<0RP?pj6_9$cVXEv7qLx1IW~XO%m?dxBhV^9? ztBcoEtS?)(;_8T%X_|M{ZR1o{!Lqei7q6|@xOUyuE7w(&6|GyjtZ@CMmxf7|KorEf zST9u_fgzBp7E8?s8@glXbwT9p!0H!F<5X*WLQ0L`ytB_2-+xf2dWP@GiWWPM%a3!iNiH9gd%A7Mbfi< z)_i{$L#3KNu53gnHJK|kV9n_Vzt4bs7!+QHJa))rkLjq|rc`n!1I!`F54fV-TvcPA zo68XjNp#%F8*8i2A(n+$R1ig21)U17nlLCbw<%CS!g^DUSXLOdBTOYC?bL=+Yl+Ix zY-*S*dEV)wmw$sfS-YOL>s)Ype^}B~E0!`FII)2IX;Lr~* zcDGaey%LZNvU5|s9A*+?uXe8BO&&B_=@uxczYwK_dW-A*JZ9uD`z?H2U zouTDfLH>5614%)WI7HZ*xYRIyd&8}PE|#FnkT7;wfblP_3)vktH8Qu#`E;TYj%!uP zSXb;QtLYbA%2Z=t@L~?kHkp!nm_j9A?n-jR3)E@n9%SMFQKRJe$*e}GVxuRSrbyl0 zJ|DIYp(Lyya_)Ma#yP!Lh7+i-!p$ASz# ziCJ8>f;}MM^KQcuW0$D31a4onLL^F{>%;}81P{G-)!6H2jcu3M7$Kx6fn&&M8iayt27a7wge3NHuXy+ zH@`FV$~}eO-TdTK(4xj~RfTO*-f*7+_W0qyC!{p1}9fYZ=*ZG&eT@ zFD}io96sO>dCzdUKXoO^y@Pv`t<-*+X~KCJR}zyjF7MRst>bRDaJ)RNHYX}QQU0mh zMsK+%&*=?ze9?&SpSnq{qvPv}JjW9jpYVb3Kl~Z9wa*|ap7JA&`N;GMYh^rQ-bxZ} z*7$sva)rX5k=Lw06=*e*Cc)ZNXtQz|+z*oIrJHS(Idh;U(G#c~Wmc_P71jyutL{gj z>jf8U`FdJQHU|weTE_5)S>=De>8Y!h?7H$U-@c^G`WxT+SxV_2tDBZRwefQo-}_x_ zdPZi}!t5N|B70ReU0VirH~^I@3Fc(!hjicpDktYB;BfM9MSNqJ(oc{#3_S z??HK+pH>iC2FH9UV!{y?jnQm6l+(xLxe8&fx9Z?Y%Z z6yYWIEc=MP%l^VPvzyohY%9wd_1_YK(hARuW9wN8Ppf4Th5u%Y@q@COvG7OXSAt1=MffT@-yxh3=83b}8uq-fgr$iI;(aVz zyh8+rVH{!}lZL;clA@E~K` z#fknK|7?8D*mtSKM5VTkX-O>?`Zx6M*c!&B$F_R9X~upT6&3Xcdkg;#4MiEE-W%cx zL)5{jC}#M%;Vr^b~KvBI+1ZC%l;7G)F%vkWiOt- zz&Grp`2Gt1p3~CL@p9im9aNs6fvpxo|9u~+6h{A1-YfVwmi?Dj;vaycOB+yzX;1Y! zsKi?QEl2y5_W*NnNPEzWD61U&QT{>vOJy$%2B^Db}yU5HUyt;4Su8g^)hN(p?CknSd=u0Jf)%}_1Tzzqtxv?iEnwvUVdtB zS09QzHFr!oHTl$B{^?=_7<=P|^jF`(lNZt>mVP0fe_mw-P-2Nxu!4h#rI)BZDbRYt zOqNSoMt)X)zL1}vpOVV*^ONQ$=FiE?Msn8j!q`<8So8DE@kr^P;+OSa9#}C!(c2xLipyUMt4ujiD zhy&5}=#gK6ky(8MZUp?5;2<@Y5dfKV0EY6fl#0I9*plO4>?xIt7UTq zcTRITnv`2(h)*a%|JVA~(ZF+aDUpn4^OPm|h>2~Y;fMDsW&A7r&9UzU`;-!M+xXs) z<3(_sVH2!zBifBKDr;f~$D-$B)8pa=Emhs9#xZJ1ev6d)3K{>kT>8o{Skaz%tMv!o zYPx<%uMGF=gjIl)&#M(2)us=Z(}cYR9fwv5-{28h1CC9vF)+G&oWs(*HzAqxi`NjT z$1m?tBYh`$NZR1BkD#fBD>KeCXZh52PhAE(8hQ$#4>�oo{;6R68l{=&leqB<^ud z>E;73+!J$ETuEhdP0u!6jiad~`E@3mc9P%SMy1IVOu?i92c%$CADT-RKAZ_PQkFR1 z2HdmaBKxpv>mt3PT6Qj65x5eQ!il*Jf3$KbqV_Xplv{=7(>_XF#cNYz1W2<_5wlQ_ z`0a3(r$?zVgi;S;f5WByeyJuTW48Z~w1xYiAqYk}R*h0|JuP%ZN9$Lv=V z)!-(1j-wQKu+pOFkrTST!gUKxj5>Gzu#J zS1gBVs%+h|6)Sm8Fx40qJRkJlg99r~2W%C(^B=EyBL#RZ#}?U4$5Rn7(Oi(VZcRE~+^CN0{X&#jH zKm_`rs1}@iG90*q_dp=vfc$IJa6LVn4lY@iU?XV^sLMm%sPJLgOjZ*tI==B8Pj!B- z(F=q^VaJHQckot%7!Yzd56y=e!@mt)D%Fju-ybhUd7uq`Kdvg<0%J7>f}Ue=z-ZJV z9W_!mg8tN7BTxuZJZ}KHvd|*%J+J!Zhi%G`nHEu-7*35sEo&AwPz+ilI4?n=L0Vof z>DN^YDD=BPOSQO(bTF&T1a(#@Y<{R zSKWl%Do(T$#|xWOoi9)wY*t6is8mEsM3E}0TR_Nq0?Pp17vUL2Y9dfE{;ozR(Szzb znpnsvh!NJ^OTx(&^;})J3703m5{GSz3(;*9>lN}?!`=aDpu0P;Zb8~W6HEKYksqVc zw7GLbE)Lk9G0VM8d5&!!WtnGNz8?RK9SH*IRSP1LQPE!xZfun&!O^BTK-$CnNop{H zI#qvzJA6{)v1fsL7iPr55$8Qb#7KH{5hKG}(cC3qrqcUMM{GR3)s%NMr(qw3v7?kk zSRjFM??Iu#KRVQs(5Ptd@tus^6+s2)IwvQZ;Td4Bi=YtQ5DH>nqmtCHoKX2?6nE~V z`v9~&RsC7)+2D6-?-gRRE$7}XrJ$KwIOa;asE#gdT2`nN#bSTxdoU9vR+3b^4g_&6 z9Aj5u=d0u}^sNN6Z#|JTtwDOCnkO}Wz1RbCrhP924ckZ|>lBl&aeAwOH_*^}je1=} zgo|!9Ai71Bm3h4-#J8{0sMc~2sFgvYMI|U_1(7VLzmU@`Ki*jl{=I}~l-4vAq#eCX zBUyC40#yxqRZKFwN~OZ?9oi6#vRLV@S=$&`TcZY57JCp_$SIQ+xE)55)J2D=#7R(v zdbbhcD;PUnkvW%h+}fEVfei|Y0Io)Hd+7=ym!Etvz{fNqj(m$Vj9)`3xGxi-G5j$R{^^r;f*GN|9=B_L&UcIt#?S?XP zJ>&V-P~F5#!dwh*qwp5VKT%_z)iH+dbtJE#twdzhVRIh~4>&-M**Qhw0uyZlAXK+K zKzAx==S$~~h%>@~>l?KEFjA+o1DYgJjVmv~zQO5ED;Vl`kkf}ij0=jnq{P_28@F&~ ze!^!TU%_v;I4y7Rh)9#JrsN^#+;@yLNnk1PKdv&SoyHlw)G*8$nl|lTEg~YEQ27yM z+W;z#pd#=B+D;D1Tb!Im?m`4n6Joe3ML{@F%1YL&tcNZI;)><4E@|YF<9hu*Fxogl zq81Fh*Hjw17G|ri2~ia!!6aB*Xjwu@QmuriG(|};E2KLKW^w&Vcq*hr+3Mvx5SW|% z>HTRse)FdlZd|r*l^#z!Mcb!(Jg=UwLp`2Xf3C;p>i3Z8_m7#idi1zqJzs}OI= zdVPAn4)t;9_3QQN^QwUY}mSK0dwwdO5vby}$ar z>iIg<+tc6EpR(Ig;wSJ%{2e}naFG{i9Y?yxlVn8_ z?Gg0F3$Ie|Lvs2m8yY>^Xb0T7;5O^&vPY*|4!8IPaES@I+f_6MYHZ24FB<9qEo_!H zNu3>3Egjd@*?lQcgaj1+757S$GavURL;!zHbHr#)N8>||$v$zL# zO=d<-cGm0)$fCDuTv<@{L#}{u*1T=$+NRJVIeJx%slIc02`uLu6o^@O)_#kZ=3(z_DA zT&fFq<$9=6N=Q`&X1hMH%z=1dEDfJ6P;~rG8MyFZnU%Vp6Yc2uK*uXOzR;mAMd|(2 z(>m1g#aO71Lw`?iM~C|JvG3{i>Ts+aHqm=}J9>Q+mDB6HRYyl)X#)19+enguElF{u zXJB(=*ueDU`Rj0f4E7tCV1;C4WkEhHQ;klln=BhXk8`G||9FI%J|6ie z;==!ezvKA3@$VxeGygR*g8xUJyYz1(AHb_1gvBe#z@{kN5C_E*SIgw06I?|`x;Ro9 zRH}4=^o(jp8T)vo>)7xJSsTT&qbLgUWx}kB zZ`GM41F}QnauD^)-r@3R@&uj24X2FVO+?Hi~K#O~x2euwIfbTVrHt zPPAE6WYK8D1H5FgnMAQ&5+(~#)R-jhFJQtCXEW(Op;BQ+7w&BpVid7dWEiC^TqGKV zhs7BeOcv&tE}A3=DUuZ38YF=w(U zR7j$XX%QY1B-2zrx9U2?MoESh0;Y(zgCt{cvEl|34RSiow3?5-@<;U82zW6)Md6$I z969M<8NMmeg7`NfMwW%0XkQkTm}DbgJEJJ2VhLCjrb$>g4P8SWw-`}9Rss8u8WI@R zKg(c{ren0mIA)e4W>_4>gr)K|*i9IFsyLk)B%{eBM$M9UNi0XsFbT23G=mU_ny2tu z4Gy6L?<|p-{0Z8tk7AY0$3ar`4>lHB8u!t&L={Ksm~vdX2{KNvyxvHJN>1@8Lg_Qi z`o8zWyD!7LJ3i>V>xO0Z3;qlOB3yFwYmPtJJb!xl8}FX@=hELxYZI7||K-%^ue>LH zY?~UFb|i4wvA-SsiD^*od$;Syy%*h{`II#0lHaevkcI83?Kj->lkfSy`isN&q)okG zf9bC`H+Zf(7+jF{V_CK4HJ$2^wKUtm^TQ*X?&%dF6`Un4Xewf_5;U8}d zHV^*EbK>m}1}pxt-VlcCbnNCxg@B{6@2#A% zL^?R*zrX)c|0k63qy3M(G5wQBL`=AUg|_Q3en|FJW$;Kt%ruqJiYPlmu}p4LEatD56AO#&b1${&faDD!CS+LJiWHP z$madSLp%R_IEkm<$i3t8x9$r3oPC_a(s~5 z-TbAFkIQ+w<&J_t_U;jQM0cPv3jhcDJFT>YmF#Y2oSh*6(Lkj`Y5{^piH8-Z1Co2ioph^U(F5 z?BeNHKR>S_Y3o1V3w*Mhr|<9iZO>O=Gp0P~F#Wdo^CQQ2 zy1%v58{OaA_LY$nJe}ga=r^0Df9IVSMuvHs)||0|-EWWRoFjA>Uq5v8=!nkQR&~Dq z$EIf`rfMo$qIaEaTC{Y>ET-PL_QKStH*bCW;e9ri#A|9O|HG|sXMFyCc;51KUGwjs z*<<_9pVqT1o?dzBBh%;1UHjEmre23K_oKc;=}%Vw=}A_|pYMJ4K%33^>#kQ>2~Yph z`OJ(jn%=ncFe~HfM`nI*`9Ejg*!Lk*<=RcZyZEUT*Xjo+3l9GL5BD|9aQs<#cdp>( z>8rMW@W@xbR{!L3A;8mjZN9Dh)~4tW%Z2Scy*1%i+nzX5*1c6|;prc|{^$)4-hIQ} z4+?EOz2wG+4!pj8;ek$J7f-*?@$kROyXSxT4PiG=|5ruFS6=<j#g1eOTz^>GH3(f6e#E{=ZKXyLtNU@BO}P{cD5YNf�`Zvoi{c!qUhg(a;KA!%1 zQ~6IGeOu|T5&L=iz4fzi{pQb>*}0#)*q8bZ6G~C4ZJu=#iUvv z$jL3N6#oUTQnKXHV>O$YO)#FYINT#Zq6?pkLYgXX5e1i@y5GLW> zfmn6iOG2TtW1s8!>U7)ypo@`f`IYx$FPBm(>1q=#&>25++gPv7B(x+XJZZLTRcg|l zGt}QaQ0+G)-a)$Kx)V~fs|ADdo~nk&%s~LvZ@cA}{Ag-=)v6m-Dx}ocmG{+jsc~b& zk|nd3s6Qx!h--4<)P+5GYLW<*U!iZ^NUdRBD}VUvd;3u zof>@$V5Q&UA|Fh;iv<_l;M&;GQaZd&_2vKs=#pO*HfHc*b1ZoPMqK=g1Vzg;$QJ*j zA9Z=p=3iY-iVVkwq>PRFDsQ5;$4f)wbEurc*zXCdA98%Tk$c9FMuoH($(5w9&`uj2 zK(dtDx$;u|NfKNqxL6^iMeeG8EFe@O@Ls8;j8Vs!=kjJCe?k@)4N*~|$rLTd7$%9a zvIWB91%?X~bn&9eiH4b`OCXisEN>Q{lb#oQ#e?EI;vw@-qkks;Tzpp;GQ2PT zPChFBSvfAB6#pfEB$($e&R?_k{>LBx>aF*F>7ggT{qo(<8>6DLm*ijj*MWEB_=N17 zYd7Bh^tYaSb>Ywj_x#5fAD1V`PPt%SdRFerRmE4WS?h3qq3yo=U;6$J`}V)}v+|d| zb8%vnDP~gq^z22sd%t=37tuL8AJ`ievpB!Tb^n7?ycMtg`Gf1LP7IH%FMH(C)U>(D z8y@R;;%j@pvG>`Rd;5%&%u{FPE?s%eQ{U+SaYxieGcLV6f9dazeK7L+ep$Kf@;S+w zw%nqtN=nynxOUU^ZVIq zPKq&3zA)DqV~kHsk*7C*H_x%&TofI%YURa6ruCDHqhgw0x->S%xGH9*bY*dl6pL0D zMaAsMz9?$3G;@OxmofRyM{61uw$tCAM;A&7+Jh?N@2;8<{#u41$l$v;w*7TY>MnK zUmMf>E!)h=DROj_7;9{Pk zNLNS;CZ!5iIbF;Uz9D~8d|H0S^e^!v!^h$$(n$2PP229h?@8;%P4~9#O#D@B+|?x? z4X36pEx)niH!b&l@qq`w`TWbTyuSaXJx_h_wb$P``0g)@+F$+g{-Sam8?>)i_9&WeZf63bi|MSq>BWu=P ze?ych&N4Uc&mVZaIZKu{pVNKM< zQ5&JIoo+6bS4dD`%2SMS=3IH^^qN-~?iiVpU`m^^K$;ylJFdA+ zzT=T;lcu)rGNeH#C&ng3H-GQafVuf+Gj>=E&F@El__&l4y`wz7xyRJ}tEr2n7^BU! z%49YMCe4zrm#>R%zB3^)W?FQK-24UOGkeU_<@6`y9q-PGG8+udPg!>SElN=48}al$ zx%qq2#Znwhuj5&2k5^^<#y{aKt{>t-Rs>~{>QxI>5fK2*XN4G9hl^-FoEtgA8ukFp zWOXBaZ^%oBF0CQ^EnU>geL?N~J^U0c(G+5C8LQ-X0>P+NiNb#T)`w%P zxnQyp^1?70s(#Vxt6Dt}Rf5cp8e~MADqT25exaj_QfDDA-6<>V0YfFb@xl&v!F1&k zvr>7<2Pw)G^Q|3Mc%Q0F5%&(I%=&0(0sBPBete`d`(xq#Y$0ZLPSWJ}bDoK{FG@>z ze~~ru{j2^u>-{waSx>B+Wq*JD@tf^sYrXH6J=SaAzz*84b-we&wd~#58`;nko4)m% z+1DRC^8O72NA2Yb`*Znm;r1JKzm1uE0vVzi)z4U{b0Ip6(#jFPK>}zZ{xBq#j6X`J z;7`UMKJ<$EwO|rE3TJ>yI3Ls@v+H7sRbC3Y5qX^u6)Q6sI}FDIQw1{cp>pA$Rya8f z)ROM3V7Hkux7xbD1q$;R7?Wf3)m0X4LBSvumQm4 z7=axJv_Z9yu+78C`K49VH0^0|eI2Z44$M};Wvjg_e$8$hyy1)`|GqGO4JV1FV>i;~R zuMpTSz=4$lI|Rtq39RFG6U$pKutC6Xz>|QT*9t798U5IV{s11o4&?wFZV*^6U?<>V zz_xOM4Fi_mB(S0#v@QZ`13YBMbHIJon7=zrte{3Ro6hV2w6Ac1u0qphRIbfk5?EuOFlxs1uJ%I6mLyZ_WVDA=z)d4nb6IefB;ZBqX zEOjpUVb@T(U@M&0>?nb-c6<8%;(klYn4tV?!<{z-+ zX9DX3>^_WnxyQufe~EDbCcP)H9>C5afgJ%n@IKmW#Xk9OfhmC9zr(%(RE`R)9Izj- z4sh4+1=b4K`vrbk39w^m6;;z1Z-U>vO|DJ0fzt!@D|CbB%hO{+v!3b?lf{RHe@EwVPieK5>)0LHHqSr4Eaun({gZ~(AyJ^BT> z8}KOL3BVJ817#vh+zGfr#NNSv+$1tLVCi)t+X z_5s=eO`Anl4%pg&aRKfLh^!rOXA|b_L9ElABC`S>ZpFL-Zod!Z2)2u?1#l2>Cq4fP zo_`7R@~FrHfZKOt-2ferV?2PCuVOray?}#&105I-V8;_88wPCHgL&I!Vl7XhJ;0K0 zVqAcSo<@6s$DcvJ0280ZyaOHp>;VjX3-tpk&!K+6FbYmR= z8(tJy31It6=qF%Z59$T%`Y!4PYtQD~EP0Tl7;#*iBz&^nEhp|8VMV1BF@MDoV0IhFh-T=)9MYabpFd(vhfIHto zyMSH)iFs%@u_Hgh{sC|em4KVdxp%Ra<>1NNN2d^`&L|96pX0_^$+#sj$XB=#eq@=w%D>3?Cp z0Ea)qeC)=49l^N(+zrZG0N5Z(tO;-@tiK(Ai3W)s1}vm&1|Gw_O_tamK$}HkM*#Dt zNX+~=`hTIs0)R&WcLEN_ORNV_o`&}UO8}1$oR0Ee#Xe1tSTezjBvuaC4A=xH&yZL< z;0e(6LBL&!5*r3gm?^Qu4(u<$Jiz{0Xb*7c60`@{ph&C>@DN}xVB&0v9RO?wJOtRk zKw<+=U_R5}f$*e>9kZeyK=)#aS-ysSzZCTVS_>t1jNmGXrR>3YizQYHc;G7Z53t~B ziM0XlEs~YHM*nwUo&Y-lHvzWXf$>p( z3-%LW%3Wv=u;6a27vOQgBY=DEK|O#+zkqpp26W;+)DL(PZ~!p*ehIh){RU*6nBNDm zZh!^wPuK(4{$+{v1GYVaaRVkihJExb_U+^74`AODm@mN2uVMUv)~6+A`WDVBU_9WV zXC$To?&_3SAz<0F5_15`&q-`A266OW)#48f(2Hd+3>)M5W96{~N{)SonAJA2915SWm!Cz<$88lh|*7$^XK9_nN4t0Ng%JW=8-W)8P@*i+M{xyMTKxl34*@_Y9eB2dunUX07xba4(=~7Wxb5P*DC= ztn+2)7hrFa%=!Q~&5_wjz>;M$Q@#g2@JgAL0PZfvJODObjqw1smdLCbuwym)1!!A? zc>zpZi*W%q03HJ@D8;JuKEOJ_ru8ytDEbN54A@;Jv)zDu zH^{69u;W_H8>KhO>^NZACYhP{p`GhwmIByvgUoDzh2=6U1T4D=^#aRW;7-7H zhs?SGk2%p_z@b{y2e|uYh=Je7IdP+2z=mx|{{ZkdnOOjbTQDDhd3R%d06XrHSrMT8 z9~cK<;TJG3fGzi8y#SLtF|UB--746t;HftmZTZ_A@Qi`Qzm9soWncloUC$fX0ARH%!{u7TwN_P%UjC4j@%~u;)$m3$XK`ft>(Mc*np@ zZ(w}^69D6XYG5gVc|S9-GC;@A4QxAL=}`mQ16cS6)C1W20p15ReQ02o11SGj14{zz z_`87>0(PD>ur|PbAK^W~?ZXDv4QTrq?*pFv!~ooge!|0l7;ukdWXW%$UfIa<0BuGi zD*^0_GO|j*!6`=80a!TI$o2uY%`!6Uk5Ip2WSaoXlZ@JkjV%5x(2;pY zmIc^6-^fY<4=*sXCcx5kBWnX}%QUiXz(ZL^b{NpK)X2*Ekq_7axTnC#S^;-2GqOE^ zM*w>Ol`7=FjrDM%Uw}!qs0XlpC;ByjbO-7O+zr?cxbq3T4`_YT$oc^HJZ)tEhqm(p ztL>=H{(-~2Y`Nr$4H`6PP8AIrY_P$G8Z@ZbprS#cii!;yY^b844H{~&L8XRPYS5rT zgUZ)ngOyfP`h_-FX~l{a4K_Ch4JuZus93SV=31$Kzjx2f{&UVgKyLizc{c0py=T3% zW@gQrnYHE|R-p}lOZ}m#j~`E!eFZ#d20Ate4s`ny$5Tzv`A<<#=*r(6PYpvmZ$F+I zht~fCdaknc`GEbnNcp)c7zs zUoB3}K{ui8e~R9Jy*O3=PW114#i{9c(|+5$hgN^TIJF90hi*g1e#HCtpeH{mPVGWZ z-4EYQ^#6WwY7*Li9Gzo5zJHQ6$qe#56q^EVd>C#03FQ7YFrk%?L>C#~3y;S}Kx=C# z6T0y{WJ1q&Ti}Ox2%y(d0|IPCqNM7of1=|!sDy+f} zRF1nz{vb|2w8`#xdA2QE*qGn1pmKex!!u;9g07?o06} z^+B7U%GU>-;a+|wRH{vWrfpIDOwGlqHu;(6pz;1tf#=-Pe5BsQ|ZA%b3d6bNOhc~9!2=8pw;-bd_HJRS*lHbu0E*zTuaaj z{9Jl3e^=^ZskV9OHhi39_MglD)yi}Ezh$hGHqW6e(9A<=r*hES>_z%cnba@sQhLNQ2EWuDtP~F z{5DYe%@&|jFTx-9$W&VXv!X|(+T=fLh01?61(pA76Dt2%?YXHo`On6n^DhSnz8&Zk zRQ|MGp39$BTbXK;Kdlccf7$|6{;Hu(v+pOk8opRneFR9nYQ^w;^RwETt3+^>EB z%Dw!Cl{Luy5&VNt`3+A&D?X0juR4`p`XqinXwUyGNXFME z{HW7V`BC?w@}subgSU%(lqY}cEcf!K7X3E!#eeZ$^ge@M2i+l`{HxPEmw&bVX{k2( zSBIhUukJxRI62>ONh-aT#;=21rA39Q9qtFAl?{xSWBAFS4aW;pgV1HD^2smT{0#6s z{7KL?=q&Gd9)h0;>YXt8DxjrLXI!3T`5ua&k$d@Z%Pz&9o(&)OO%KN(1(jd7gXcZx z;J>{zl^(6Y&kNmys$BVdm!3)cKvU0R9>w3=3YEWi3c3T$G-6jO@pnPx|J~)e{J*u& z#-8E-9fHdLy9Sm2x3-CS13E^2`GLo{pMkD$FMseZ_wom~J_mdB1mu1u_8TgBC01X@=tF;<)5x< zrM_+WHKFoPPebLO-h#?MUGpO7@8V~L&b`FqLc4?u9fR&cSD@W5rM}QLX!YgPqXT__ z%CEikV*2f6_^aFKuUC-&MU2x<{I=ZhK~*pBmGJYt8(RHh>I?0GPW>MFpu5ohHp;sK ze{nl{_$tHK3eEg3{rzflzYp!@UVii1moPv7zIiVHdHGAxyB^y2C8@Oh=zZMFkG=qv zAHA#tJ?l038(vTSUxxh9PH5#D@Yh18po`G@tI7Xz^c>m_mH&O=<+LaMchyIJ_`%Lp zTiF}U{i!!m?kgBC*P_4BIVi`h)Aa+$2_1lnPk#H;uVg&@G2`p^DDTbY`O5XQ-xcH^ zqJGdtsPbptioS8b4b^?w+wenk-~4vkr3*U(9qeLV@u%i}$2;i1SD}A5AP=-`#D8AY zWoY3jbd%?b7u39(`FqsxO+y!6&Ajp#=3a4xu`3x@@5b)%yc1e=CHi+0?c9xBh4w%d zXBc`7>t*OFRB?uiKVW>_LOr00GwegRppt(FI&c+uA2j#<&{^&qh(qXo3v`2f#U)x^ zOZ$Epy@e_+u@CK;!fy4ThfvnaZHiA!Lr4CS@_D`t70#NWMfJDh-|t1gpq;NrkEV%R zK^4!)ydl-5ct$_8{-d^hXx-JUkBDoGKo!?mgEq}F{@%d&{u`4|aSpGKc?;SFRh(l1 zsyIj4AHoOif-26j3hnun;ZwXL^GE3IJnaCj_&f7lagQDD75AvUhWrcU>!;pO)n^5| zcnx~>_tb;uo&R8-D;`qzM(ocb;}P2VkLJ1JBDL2tzbrAH-pF|UC-b}qy2riZC4;>0 z_$+aTH=!@kQ`~ngn|uS%NuDo%j`0muJf(2}JNYk+6X@)U$+s^0$LReRuyb#w9sd

QfSKOuaI>sZk7pl0+B2;mg@?@msd-)p-QaoAfBW}EO@+WTo}-7xbV zwEa(Mhx?iDp^6Ws-a)(ljCl>J_|ODY@u6+#-ag|NzR6!;AE1gC)!xATz}BM=s(8^7 zw3m31`b}}8vJvb6aibyV`f>cz@Rboi^8S>5hpzBkair=$quq!jHT*gKKpbfTsyNa% zRB@#0cOpl5IyC~FeK>yKKcjw+0FQgclPX4`=MYDLDxS11`p9&u_Al7~L5HA>2T;KB(eLOHjp^%5G$xM|`OZs`%0pRPm*HKRdMY**vEniZh+&UU8=3amGKiTpJgV~j)azx$3!sWetwO6_Vel1~s`>!!PF!jbs<_lD z)axSda0`5|Hu)5v>gT@pN^{=~Eq^cL2)c3$`zQN+7rM{A;#dnGq#dE&hu|ZA)d5u; zYX+(~R_ep7x2{5Nluz-jR_^EifVdF%>#rr=0#$r#is!Abr~cf}zk&Q8WPbzghAQs0 z$$Q1U>Zic{Bk*t4{*mWX>|?G$Fa8gD)}KywK^LG?(B3!l9y$#z`%Cbk+we8KDV=KM zeh4}Q-GMGZ6+avN2<`pH=p|I~v+BQsUWYzI6+hb+C4Sa;8~yqg(@uUIjeEt>Hld26 z)y`n2hrov_j94+%v?8)2En`zn=+Q+@(X=_l$(=vaJ9ze%8*;ZF&{y_Pcpv$3jN`Jafbe#W4`_MczY4p^E=)Ko$S1`80BzB0p5|zeT9xf93y({z2QJ zivP_*mo|*NiUSsXhUeerJ+$vT)c1DuXbU;HS3GbJs(4_-5_735DEEpBo`QNm zGS3ws-1;ZR9klkJY4`s`@Bb;4R-ABvd&LRYpq>B4_~5zXg$>J;|5NB^(IaRdRPnpDbhbvIU53BzLIiMp@#SgckUHe8J#Sxo7&p0EFI0D^;N)HrI zT;YBwRpj3*u2{B0yA}{Xqx{ywqSQK6amEgwmlpZ?iu#B*ihl-L`vvAx;*G;l#T`#U z6?d%qSL7nzI1E+X@f1{X$Eq)4uZTMiLlt*ihL)8Q|5`zx&mw*ZRUEQ>mHI$Cp^8IJ zLluYIfhrDJ_a)l*VZ;-mibF0#Th2CeDIQt+W%`|XWG7Vd$Q9@g@kq(7xMby>*o8;J z4^>=p5vsUk`5Ns5?S?8Yxd?5^7(T@(OTU7CJkH!JPT9`A;*^un@+bJuixjWi$?ZVl<4z0MPC{=woc7tCc=z|V67Nr`$iXY^e#QDC;xOYWS6r2E|F-ovgU@rtS8GxQZI#bA`8y3w_1ODfd$0KGpz^g8rPknUf~MA)N1(OP zVcpa2iq9^Sub%j97tgz)daihFX+c3-*$d6R;Eedrqb702!6 zUUA%c=qYICYwU+!Mm+m?LE9#@i~E6>8#!m8Gu$iA+s^ybQ2vEbTJhd#?kiqN{FnP_ z=n%A;_^;}#IPf0tQ&$j2=6MJ7G;|rN_Z!g0d$HqP=Dr!)$9>JK&|~N#G_{dRXI>3H zRPp4N;)1pXsQ49Ep5$I}0>zaf7C@mdbg15n8`1fAk}>+6Zb zLo;unUB87K&@Sl0)x6hzA8~Kp{~>y_iN604azm%DF*x;aq`kObc$0r$q`i%`YOtIjHD zTOMT`z*qVn<$VV_%eaP5@%0_3;_EFZX~)kp4xx&# zmp&AEzk;4Yr@uzO!q;-It=|UYnfr-vFrK%d-=zJaWn0u6>TR2R8EEx?&`&=!_q)&@ z?yG-9|3VeFUj<)r`>HbZ@+Y(tRB`)L(Egti=O$^SFP3smv^x`!3Ct)z~n zcDe5@DZW3>bH(?!po;I;{Tl59t^XeOs))Ed_r1rCrG~g)fS!g99_Rfw?~9M6 z2B0bLSZV<}4?PX-e+citkN%eM9=ZcvfX5P23 z|D!%m(YT?7>!LrQegB&BpgKR%UQWMpeqtJ`^Akl6XME2gC+|x?0WNe5s{YiuieBD# za;~EKzZlohS?+bdVh`$ZzQWH3?clx6TQofad%QsVaj$b1^W5v)Md>+=f6iTuLUrz9 z2dZ-ybrtCQKXbl={7s*wAMT^wJ_jB&!+8wh>s-b#`E)L06}tGvW2q_LH?AH_Z9?b2 zWb!RStM|~OFCR5-JPb}Ti=bDifH=3eJHPC>{14L#<0^If#lPg#H6 z%{h%nG5$H<(F@i2j(Obyw%xdm;jKQ!{5g3j?gvqSx$qfk=OF0|q__5P8` zH)m+-^nX$g&wGB1e*Fx+ffiL3v~B&wOq+0QWkdvhnDGww<4$r`&h^kHPDNu5(|@xs{6lVgCZHdJOX{=T|zRI=?aw)%lfG zsLrnxombG7Nprr0{KHVyN9S2Od9L#;)6m{y$5V5>Ux98y%Q)XcD%3fbE#B`#D}Ih1 zbKa#qgI=BhkNd{b!AEPQLB44Nh4(`FO7LG4njv`Iz!w z$1Y@!r@Em!A2SbKdHnIz3i-C6DqrViY932I&dUr#2RJWtn)f<4v#NW}%_yJF&y+r{ zpl$pD&guLw`d?$7FFo0({<)eK-s@b=EL7)eN~;)uoU7@DPCfN_stdl>i;t&fpfk`- zCuf-E zUZ?U2=n1qPs`EPIP@UJ=f$F?Y-TBy?cNrX=+nMBkX_9k4;Fo{MJl8p%o@(%*i_q%- z!#N#(MNQ{^4&}OLQ8=0mbo>-7x{0Qv~ZJ#mEccA4@DoFQylygH+o&Tx4 zfbluYc_!`~{>J3fd7#?gU|qyHph2k411&;z9%vt`^FZwv(hr~H{RIVSoeP@eUgv`L zp*k1TUdwn}p#ONU^FhFUoJ{z+*0Zx*D>{snyKI#jsZ z&|&VIKaV`ni52wbBHHZ>=KTP)hWlk`C$#KK=6Mx#n)|wS49&U_jAxm z?z_Kf?pwaaxhU?ppena35I=+6!HTc5Lzfw+hlt|3SIX>hGf0(9AaV zxR`RkPkGR_ACMn9u!EfS1?j3Eq3_V~|0F-OYZpEFZGPhIC$tlE9y$!&zYjfyR{alr zPeZO>@ElrSSe)8|j-)xKbxA>b^dZHme&}{dacUV_d9pZF*HDmdsVF9{TaccF&O^7L zs#n<~;pe{TQRcqG(4r-KU&eW``llD9buNsvCT+`{3)|&h=fm2$*ZHtXsLqFNL0cbN zoXR`{`5#xD>VXbIXP~oHK7LW=@ur+6Xlo4FEGzrp-bGKy4>6^w{o71`?VJp6Bj05 zJNcnA&^2i1OOUsTetKzfsslO&9fNLnAP;onWyPto=M2M&!H8sr(9_3)yUIakly)2%Lg6ezW$Hk zgL>B>H?*|BIMw`IkjglvR+=|{w(6f zpF@A3{Oi)x4AfiW`AOy*&Nt4S$GQ)Cx(dH0=Ns!EkDnjfe*ylldwjg2^7Z0W&68MP zLA#-2(3uOFr@zKI$6D4^UpM*Mq4Sq8e?j{>&oFy$ajLVOb@RV7K7W_>!gq^P!yT-P zzgL`Egf>9;phHmglg?AFcH&prMqggRybkSw>U?GSEAbb7A3UheS588Qeo&n1{XOdS z!{XEq)Z3w5x>zqm%U^|@KZ5=~{+*v74^-zbi>|`2{xkA-Gf(duxpf|M{txg={h~P4 z(8D@`=*T2=9GdE7Jy*b4O=we^Gq=#1A}^JBJ?(dlGuqH4D5*&2Ip_QFD|zOA4%+)B z{3rI_dkANHZ^9qWc~9lnxz9@c<~sM;1)VSB4Cj>2k(l>72ikr+{zZGQ^Psi(NgK}Q zEbZ;o4=S9&a?X4H8|Q)^VeSW@r|xF|2lc+nI`SOzyp+R`J>O*g$$8RV_TL?kH_vsh zbeVgdE3IOmy!t!l`7U&UeOljT$j!d0x7j?Og$}S^DC7L8%GWv6S=O;r&*zLh>(M68 zqYgoJF13Yqq|T*|Lq}d{@~=XNSm&L#_d2h-$NFoS^Qywrxz$521nMtGE^j%%1}!}8VZp;A+vuPckMi7bm;-V025 z!thfTUrAwvv*|<@O3HlCoa0taA2v9uLpDyotK{)c>^8rRVi=&|1nP@UMOS-i_nU$DiAD zq6D={exq`FxrQl!q|>y|w5!h#gYvsN4k*8*aI5sZE>0mw3#QQ4q^FeC$+brri~5mh z(y}Y>vq5=F1qW5QB;8z4dLDVQiv*a|t_wZMTw(CLoX*@5;7=9gtB~!ZSZ$0N`Eqj% zB~|?@gKI`R{d8l1yHbOd|X{XxU?K<(Y6 z<$Iv^ZpHFFSbI12eIvKp*H01KbcnsXd%)VeRv0r}d$w-^TlLVta(xq!vRgy#F zp`XjOYcL+Z^AU%6#^YfX++J(vgYj@rfcv1wLk+rFdaG&IU_6BQOTUWoAl)3NoY{3# z&bIwWDcESm9A(^9FuKWk(75B6ajNFKrkr5hUHws4&aZ6TjegIR@j%91wbkDT zI_|nG-vb?Yqn7W%j=RyP-tW&NXE5$w$=4P9b%q;v3$|~9areu>CZ6%QbL|?8ySoC+ z2Q}_$e`xwR7#H*TmoeWBOL@2dwHpsd8+RkD297Z9 zs(;gz6O6l8e9V>epvPSYHbm>#!HY~8r}^xsrjxBnbE(#DC53aP=e0)W)31G9tLgX? zK%D(pa1KRA3|m(HJuzgWc}4s zRC*qH!XmWRw96!aeNly-+xahpTN>i#Z;Nz4knyTJJiq?n&D{U1mD8_3xE=eydfY13 ziC%$Guja7+hqdeULG9W^?+k+5Wc9%9JM!rP|BoZF-^*+_C+$xgtu3vydB84fvN~46 zMzgF*eQtV2d1P}}FE)B98|tUXx<`B3lJ58hN<6apGtV}BLp~FKnoicXWa)EBA-jm$ z7N=;^mZFA&V_Zo~cUsMU)NhjC{f z47I8AGO$mJ-iTH8o#(2sy3ojHCGX>T@M3UAo;~t6TmD|38~%6Z;g7Fjv&Hf4UlZ-W zX7Oen-emz^uHLjn6@^L;#_t8u%{2*b)o+=8t>v?l_w!)=n#aEO#>bU`Z}gioYGAsTp@t~ae+M*}g1UcOk?r-h5>vwWrAK>M-hxX{<$niyu-bxot!VtgI72ih4bZUV4pg7iQ$({DtY<% zo1^UN@lZDLseYs2l{Xl?pdT*IBhL}~LB~6rE#86(A$i3CUjF&6urc3s=l5@25%$sI zW?Z}775Etv`giEi{%uD#jvJ-=o^I;DBw=FRN6b9zW2I6janf1$xy zckOq5fRo3cnC?Wd9EYOt`oWvLJPYsYGsjy7uj)lvcvqYmp8B_(KAnBB!Rz)z#`MGG z0p20~JIqEq*WoC8(+loGhru1?Udg*Dz-@KD_L4$uc&qPew^rJ$v!!cAFEjkj&JPgk z@gZ{U(H*&RslJ4JQhTp4xGRhmB`+P=iKYDh$n*@A<-J=1$**ffRzG)w+x=RDt2su= zJ2$}1r#IR$<>*Z}ytCj{^cuW1-+K<;i2(0Ne&`;+^Lv{Q_80bEYs&6YMU*0IvXa0a zoBG%BT5ZrWWO(Y8Un#<)^=D*xVP5lhn9K3$v!Al~4vTXntn1Etja}C@1DN63f0L0d z*mH)qCO(%I#-^*-q^~;ma_!i*8Q`;$cU_>@aje*SI|k+RX`C!_l@A!%f|yEZU*mXL zF%`+P&$aqy!;kOMPhJ@Ri1QbAM5p4C!s+O(i3hZRv3H%pTXlG62ldOb%O;$1Dms5C znnGoB7hs)8Hw$LhTMYiP5jW-i@>aM1&x6k#9fwQAS9_K+_J;n%;Okz=yCcAlw+Cy} z=*mR-Ti_qG{F`DXc{k_bkFVcTP2?Sgzw&xpKi7WO1^lhK)6gFFFFFe`Y`{4M&V(IL z3l8V{04JxNN(wikZR8J3;Z!oWdT-0>&jaPz&yh!Z)(y_++YQc6t~_IT;%q1euR@dQ z%OyO%W|XS_Q-h}%l#&S^MaszcA%*8kAUF{r!L8Vkmi@c(KNR z#k&lSv#YdO;CohYw-(mBQ=PftVVofL7Lm>=<+HV=0nLXqGX$)~%qkahH zk5=n}@N-zoWnO(;1(uYmvAT5RsL@B58M2Q zwTt8$Cx5TaU(LOrvidc$UH4zBoK^TbEnlzUQa{!PV$piFzEs9Ppnct`2~&^BFn^AH zkex2k)?Ga!{*b$0ndJ{D8aCKn7TddE%Gv6;E$s_?)vOAPeBJP6EJiTDrSo89 z&2Q4LX|8oU9$o)>_kY;Qkh}jP!k)A*+5xBh22+<{p8R}(6R*n==1Hw18b=J?1lUU6 zhx6dk?%_Nm?_K0BdShVxwbUf|)zr(PnytX|2v2~Qut3~*0ZZz!^ z@Sk@y{&vdKxEZzl>Psc>habwXR~#saDOQd{)Y!vz=C!^tBiDw3<@ZM)04_Xg-$`(a z-feL99nOaXoS0oKDP-_D%Se}tU;dDZ9~*l*_!)NNuDPvH9PAY1Uq;e8!X+=-nNYZd@eWox+72 zvvbgzY~`;Fek4v2_yaKC#v5S7^w$__Tkn&GXNGa>r>2u@iEGaEfR6S=*PMQKVT^N? z{)@$7eE6y9t3S0StNV&6XFk-6K&}%0pDdEG8wp&UWGXbla5o5}nf;@g35-SV}Ed~q9zISVW#dDKo- z=wRzzM*iWDFVaro8Z=?tKCwM(t=91ogH=lVD|u(X-;LWjbRawjS;GGuJdZ~adoBAZ z-O2n_7RG%ON5)W@>X%ur-XEBD+~l*8cV~dpmlTIDDICXHmAz9mzChWbEi0G#ItF;R z26zXJ-xVCtiQ~5uOxiEiF=y)A>q$y*uM2R;4mm?OfZ--jvYnM4%!A*(VDL2$DtVU# z_`3(#om%V;@1xjee(D9I6n*PkG`Oc6?gdA|)j@|G*pl=VCNt-;FRrf>TC2@{(Ucd& z=1v6V<(U)E-7KHYAW!nRhWs;~Ta+_k?ccH~p0(4>PXF?hb2Q&j(tOlR85&ouYo>0S zwoKXFBpuaRk=&yqhDrUR@l*S62CMRH3x?fGAAA;d=V9oPewaDy*lyuU;&X$mHQp(% zjV)7Ny)6#eWA>_~Jv3O+Eq{VN`P${yo^*hf;aafRoz9+I{yx_idB!k393R8@>EaKP z`ayM(Jlo$hHe-m-eo8oppd;_${?23*zho}`p=pbG^H_6dSFXQph{-VSZxjCn*U*FD zUx$D1LGV}5#yvYmey97%@{i=mUq{&;@K@XR^K5@fkK<;4F}8M4n=3`uf_J~b>VVc} z8Ll#`13ipQCGXmCrvqyT*sn<@UfxG_8pjX({ajVwHTCRrnC}cQu|kn~%AMEu4{4=^ znP{cdf2&+I+Xi=&&r04c0d9OuWch8Abg~A1wavo&JI_$GR)^r&)wj9Er;cD4E8ot^@I@xFQtKI#CPGpY>tq(&N zw8`ezADMP6|B7Xg)l7k-{)Gi{ky?u zCGWm>ySfE>Xne!X2pR~|gAz{+H1aH;%ot^WY_% ztr>>D;y%Ma@8rqH-yD;N8ivw|Rqb{Pyg`e%;PCDnbNzaN&a#$D&{>CD!+vGM;%1!Q zekQ=p?TaM9-|P##?Qz)F8=aJ5HP@~$`tck4*F zIun%nZ-X1?<@pET9zieFZ#omSmyLVxjjrEv+m|63YX&By=)LMIecuAN`Ti{Zx-r0w z>w6(O5UNzpz!aX&1kGB!lxx>lSsp26}Kk1IKVq#O;9+Ruq5!~ z+KCGV%2IiIls9PS9e&2gPu_j+a_ziZ(4A)u5`M|=*z+EG?D=IS=?Z*ptw% z`3J-7!N}m9Wc$CP5o56}l9J?XSMv&?5D z@5g_BNIBT-ta7p+2&ai_o2$z9*|M{>9}Dmk&Pk^AZMxV&N#Upsifr5Z~8K3Ps0Y)GA=!(`b4;K2@YzpIC#S-=K9++!S!}J{ zhPhTBW%BFZPmw*FzxG94!znTREFDuHtyumJ!;QfY$9Q0;cqiMo$JP8;Q=aPTr>ye2 z!XnIT{?^bAo0d;9`zed>DCb1_!6~gW=r@acA}H1j;eRkv>J zx6Y*~c{h$Y9iB_r-wflAj_q&6qyFgSYP7l_Y$fm704u(BiTS$B+Gm#h&IUM}?L9jW ze_;$DXRXo#);`y^l}Gkn$vY<>UTk@uh$lOD%IfQYf#W=C9=u?}(5@>7&cuzu8>=w= z8=SYe^9I-U@%G9(Z=w2az~6kX;dk@8_t`xBhn$lTo?<(Fl?JcN)$f)(c(HMoTfa{D z<%=G&csetrT@HfEUdbE*OV_V6);xPQ9*oJ2zKYQ99DZ&@_&PVxaGt@Rclh_dBf0dB@mQd;!J;xs__!5_2ux>xdU z%Y)CpCqy)R(}p6}L>VJr#?3!(5BLv>mG^4Gj(HXJ9RSsP9NZCWA9s*Z$-64R-95wy zKd%T#t`FYe2!9X!@+TO%`Dt)Jc{c|5n}^`fflr?JKEYr3fTTIAoqfpU6AixBRZ8CN z0sbNFqLbda?V|BF3vSm12DjSb-V)&Ej2Hj=AZZ;-566f5o`rA!z^i|nV+6EHO{`ne`9G}sf?PJx}%-X-a}SX)ac#ILctTYKjG zd+_f+)$osp<>&ek{PN?AakYz5WPbnjQ|KYnldC0`%7jhC4EZHBmUI}jcy(ZVHcBb#6C zBt2TUIA_vMrzOX;jT~Al`zc|(seb18R^Uy)XWeG6ur#9gpZm?MSU^LaDI-`*e(SAH z@0SwKBl+GaUWKOLnSDQA+)tScR+mH8j%xw=NXPi9b@6)c+e? zQ_nE62m7@vf^w$v$K4d=;dLm#``VL_TZhc;muBIf``~apk#!VYuhHQ4xOTZ`$hGzE zA!jobIU`S0Xg7-`&j+=Xss0~%mMPQe0%J9ZqtNQX*>|$~J{!%6?Ux;mM|Nb8tHI{4 z=Chx&>&oYx=I5Xf(TbvZ1mAr2SI61bc#Q$PNU`KtsruNu{ zx6|6dWhQ%U{+9l0JeEB-N8f_6so%D6Ax`C&ub}LCCVwZN z{S-OFTi8Ug))y?92e=w+{&ACA<144m{`V-d+ddcleJ@$N zxui3#|6=@H)ki+>U0;;6S4&T_L(Ja7fv<77$yIB0tI>|je7c#gpyi`&1g}!kxY8I= z8`WApbo~H!UY*gm=Hn_ObhE}#n~b#>o!RoA@+Z$2I_hsDNmmxR%2rJ~yS+WQdH2L6 z>3)t+#v9Wux>R4K$XxVoV)p)6>fm{XYtGg|ad{>0mOpl5Y3Pu%_xjOO^5m{NlG?u= zY{@gx`m1fTggXB3`B`=Bx!jbgwpH@(9dLC_9((LBqH2|}`t)^m5Ug2SA7=->TLSEa zxrl{DbmIcQ+GUxm^hHKSf33m2cXfc1x9=6A^?%PI-u?dKuM><)#?NVsyKTpkcS(SI zSpN<)x)S>r=)^Hr$8+^Z!51ZVQFY)}k@@Hu8u_NYhAb%_Q zw`~525ItF*{FK$ECBa%Zce|?gRNK^BjD3Tk zdJpB<|4m9|viEy~Mvl<=aV_bp<7?!zpO{CYeCTS7PdXu8t$VrYk3PetwhX?* zp%6oKpRz!new)8HDSy_!W}7^7Howa9Q>4GM_cij9c&{+>G}`RZF?Yzm=Dm-#eb@qT zhO7Fe2FuCno#=Oc80fLTuL*#YV>{A2wa>~IOyBhLS;_nSH4ZPqm&HOg=E65eUh9XMi(wl<)Rwe5PotRf{vHdy;oWfD^2JeOH}X{JLq__i;AD-(YnukmHho zKUfPVeMd22{5z3$TT~~bHrLjA~<;wSLohd&y2213NbE7cN zeybK7weQ#!M;n)94SXT1mggw*!61CYU8al`fhe_~4D?3FwB4TRpf<04mBFYF>zuRB z@YluRI>RqYI^4lk{c6KM}w3aY_6XoKD(%2K(Z=D=G^~aJ?O9f`+1(>a#+Efd*W)>U%Lmd{R^jb6fkG$&u`>2 zCnvn1Fj7+gy79NuW9k)*jp)2J=N5hjNBye$*S_BHG@8fK<*ArGXA!Z;B|evE@6|b zP|!^FcF$k$C$+89*~T}RcIrkRKP8PF*1O@DH`$Ufhj=C3YPFclMJ6w8f8cpY_NlaC z_EAXHhec1qWBy5V(sREs>F*@h=3Qo9pSF8f+5e=u&U~e&6u|gyp3=3vi%f!Y_m|KT z+0RTJ6UNlLA8Gr$nJgKu{XaBq-Nk1m@4nZ${*JGK&`r0Qq0H!wziyHKl>L#x+V_#b z`c#0m6w~t({g4-rjD7~|j!yXIxdyG@hJPFGC-3?IXYP<#{vM4+p19(>0!BfTFMZdp z)B1>SD|xRA@ehezbQdX*orACQA4!i3nVAxvn_~UF$~5w<-D&i~jc@OluXTN}d6+yX zDS49Xn=~Gr{p#;Gebr}W(Y)&Xo6&W>Ue|E7aIILL!)%`N4|4Ur(UjrzkTL?FaI

zlTZ0)xjL^k`Fr^6r`GVSUa;-sh4f^DYx9(8H+Oad{Jb`*q;Njl4rwq?=RP#@XiYHl zRfFH^3&Wp?kD88cbUt9o-%I|*0V98d%ikT=mq8lDr(AnP?C{SXZpWp{%5W`RZLmhb zRr1dJ1E&-DZHE4JPJ$`WdPg>Bug_pIulmWmzsF$?C7k5`InRe+bX<}7K6&L5}(7Ac%hjKvV{EV#YdxaU52+#=?HrEJ_guX6p7D=+KqsC-Fl zUbSl{xPx|_uBj17-rED*Wd9+9EABrueCqF6@b)d0h@<07VCShk=?wbapmjIn@m41gggoRu;HA2n0QCICTyAY;XFKb;JlJP zjhQ8j8w=*EB>s4i9U8jZj0e{~*>xz<^3zqK=|p#UV=9o;KJ~m(`;=N6g#XE;j+1k- zxm3Ro%o8J;$x;WA?v7jg8`z_cc-t}roK!xe`26;~i>)$vA9f?Xzsz z?GQbZsw}FXQp67aq%6x0wpv-rZC`oc>ULwKFaOt({a;tjITq1LS^ZpPcAjzHv+zC^ zU>>}#F7a1+ISXsIzDVlF8p=~WI|q%syvwH9|T+MGcWk?^CysNrg|0miL zXD;@;6!8SSR!)`9b_+<9!i#haC$lcZ{oY)Zj>0{1ll}gJmNxlz)k< z$?jVuho2(*IYxM++t2C!F4u;|*F555)8+fU23zr7!JhHZPW zZLMbj@+lV z#5c-SKK8))YT!}7dgF$#gU^2I4)+D9hcWTIkZgNgT^}&{J0s&KW*>D%u`e)>T7pFG zn3lpjzc|USvA8v5aF5z%;v0sq_cp^f=A-e4@gJ2(@#g435A}~yWPkFaqAdLzwmPHw zDtY(6-04hgAM5`PtZAW~AhOfHGTWas1`5AQKCC9!+CElfKNgv519kED%&TB_So~p! ze{&xEBlufne?~1{u%5Xpz{|BC`cAN0)8tu0wIf$QxSdu%mz=uLT)(75?1KgbN5c&TDBK#7OzmidX_1lv}lLn{V#Ut(vaCURPW9J`+NDmds ztD3bm65c#`<+mEV8P|U22kkc$^BpMw%M-IBz;x)hAR_m|E8;taz0YE1-)r0`Thqv(8aWNmI8a{4tdbsyf zB+gP&$oj!KTkPu^sM@~Q^7Y#1(e*?4|0tVID1R^c*FItUEg?Ue6wLpUYm)pOpEUW` z!u;W$O`)+Yxf15vZ|^hA%1Y1kbteN(gVi0ipOW{*m$ z{?{IckMiW;Pv<<7jT``f^=}Nm;u1>Ul@a{>j@CYGFy419#o(*np1;?(+lO z*cw6qx2iVK^<=+;8{2(W4UYXx49?Xsr0U`A>$g-m+0%u;-&^+_6c2Z@NPs&*spCum*)^+kR7@N#1Swu&HGPJ9;BN;jMCw zS-delBzZUH!NXh}0O!YIv?g^nDu_*95#S}v$(vZz_zwtkME&hYPQ#R$ zV%hV)I8`w4m;%kc&hDnZXk?I`@l&4tzurk-r6Uce4UgL+$qq*Cr}4`mt26%gH^&m} z+}(Rwc5Z*!U~3Je;I^;Vqz|X30A2V zm-*%h_0@Roq)$&-{6z!L8Hq#i2ag2r`@R;zul-k3|3Lmr1N>Nj8~yKy>wm8M-5*Qk z*nxG6JK*F#H^7azE2#}&=Dum?Sm-0ZKbC-4gx@Zu*QyT_v$l<*rz)M(1ujj0z)ql(2lz!9Vl&H4UH*Ei2)7SZ z`;LNNw`1@dWiv_MJpq2Q&ni8|x;b}4&;aiSc=fvmFXh%3-wE*I`^$7UIN^_JCaup} z!PI(b`o9e3DL(s&_(paN%zT$>B6~86-@`0@?d7Vv&tNY5fc(jeITFXn^SeaC6js^1 ziJ`YJ=KZAarXCz)8!1Zcr; zjZRX()PSY($A50xa7ExskvLuS?3eOOrn<9!=KmhGbj44R{K2%&i-fv;;`>>3Yq9-T z>BjN5Uf}G_k>kaZL;Y5{XX-KF3q?EK5nx69dKkhizg`RZXSv!fe~**p#(+PMo{neh zlwY!RbDyjCr$(MNH|DPm@YXnMm~2~0(xvRTcpn|_X0&%oMr2$3AA_s?t&(?BfXiY2 zJh;Y4>$i>a9{5)~%!9wZZ}1m{&-0xSUo3Ir;cXr79R_ZBm_Ryl%GReU*_tY(Dw-G z8Z^&Y7u8$qpmsY37JO82Zwzqb?HK(cOu{K2;Vpny;hBDE=ChJ_MSz!V)Aaq3xZlj@ zldkK0>*fiA7ud*)1H9I_&SD%}L!DK-HN)S3((niS$nyez;)Z$FQmoOiV~KG^;ZA`& zX~$52d*AQ4{x~DtB->U`SsDvvA2)lIAwK&l!H1pwU4FHnWL>mlZ{Bcf?cbHXCrkRx z2=R||)jrI$-}BxS8hgbHu2Xen-h)4Nwy9sQ;a2|ydz&L3Dj9W zjbT3>@Lg{Izv@I?0bl!il8yc8z1GGy0+`|2{A;6kuJ64|nw*XgWyK5`d^|GKzAvlV zL$mXcQ-V9e#NNeYJ*ii#{r{Jp>|e6x1W3-$u{J=@6=w3pxBEWg|&Btcr> z-Qn7^_vxrXexoa3{SmkzMw z`Vw!?pcd|&j{0PkYt!PToF3ox%tK^Fk>aA|$SS#nFJJqR9nXt;Nb+tu1fTuY5#aqf zU=;ifiyz4U(f~hU{m^lw^}`mp!{tWL)aFXwMFDQSo%^F2meb7sIIY93?OVKI14rDh z(Y13N?+EdcVeo1nZsZH%L7&NkcZ7J*Dey)gVeqD$e4omPhgUEhr#Tgtp6cxChQ(WQ zcyG^#$0GRbIon0NWcq2}}^Yh>x5?_*@mlttW82KjLIJ*B* z*S~qj{PvL)UG3Qqe#;{bzShi2-e&{+xc=xk@geQG3|{G@3|?SIZpw!j+5O~MAD3&- zpN-dcW_az?tsA`BbB%n#9DR0xmoTmmcfP5g7Qr30xJ~GQlJ~`D9NbUoL6*#iTO{>U zB|d<@#~8UQ7*k5#cLMyZeqw)cNI&(0S8wOd(*_P-Uw{{DhY}rca(g4cpTZlls$Cbs z+q8JKPLFO&#xr~EtUW)Y-Jv>PlImSfJZjc{pbd-^05Zb`6Rpx z^WjY$*p90AJb2ZQGkBis_Y(nLtR4M&r(5XrXiak~F5FU&x&HA6x5KsP1qa|39^Rh9 z?H2A64DN=*{rS_Kp5}>*X=~T2!{&G_ql;p*Zb@3-sqeI&sd$np%k6Dh3mvrAE6TTr zYyzu?YpBNHE%Vt=S+T8S&Bu;Cm;Z3hve6B!NaWG%S!9>?EsoYze(DbQ5P?S%fwGJ1rzj z^;YtJ{4_UC68r)4iN9SUo!z!$z{%v@8{owIfTj8YX4~J_Er7Rpfsre)U$^DK)4%f# z0exLK4S$ig_2Z&pAg@8SR}Hs1WX3+o>hHyUe*&e^ifbGF@<1WNLLSf4DTY+ybi z;gg*5X=UuVUgBQK`%HkBD`)oqB!xFZ&Kj?X^s4FPVvEen?pl+#}Wg0Wq1 z`aM{yUl-uzj4gAn0YwERT zS(wjbxqYb3uZ=`u4wAd({YLhHK5%t-+14?_KO{LHyj5&bNifrr^Y9 z%OyKkb-9sKGAnss3^2lTEPbK9Zy?VkSH0zv?Nst^3;5#o4Aut5i=}?w;A*t|DJT0i z0e_zHd^8`H>Y?`7wD=?3D|wd$_;LBzhK3Cvx8>p!o_tmnt)`ti^pND8$b*Nw>M%U* z4>A@{w$jJ@{BOCwi_2Fyb|9eSlMlY(MMk~>1IKUp+|3p zFc*qX@~wh5VDW-+cu5|-gO|%bUM1st*5U=ev=e#o4)LW)AA7;`+D!ZHy8ig(Q(b@L z(Z{3DFRS2Bv>W^>%2V=w7~p61cgzyxbdcoL_pP@r-m;Gd-W>s69(fz1>&?V{gP_!3 zgW$LPu8}wBuWJMRxV%T`uT}7dEM71kFUx~>gz;F(+_zxyv?f&Y&dGy!h)=x%-d^x3 zUSjI2dnNCCb*}&8^<}LVj&C#H>wnjlTmD(Yj;}A^&s|3yVZA3gitzpPSlqzQUK`-X z<;t?NnyYl?yU+3~PORiTJK)ciD_zO@BeoKa-@-54NpKfBOnY_Gw@TiX0dC^@=9Y&{ zmXxIR?}el58}l2e;+Nj-ftz}z!R>OmR|mLbIp6;`{{(m+p?T-l7zVQms3z7FWv?=L zrLO*;3h)x@ub-}o*5A~36rAZR4bCuQS;-4=@}5;MNpFD{{|7(DvtwdxTi~w!zQL_` zcIS(Md^zJd>)U)jl-f;azL%~tI7<%a_WU^JXA@G$7I!45UTU{N@VfuN;O*-n$@^G< z*OeQyXHw~kMoLY)ErU~G*Flv|u26r+64#BsqG*H7YuBastOtscv`&-ES}(Qs+CGxN zN;zw;*dU3FWRqQEu8Zr`)dt6nuTQk+UgPq?LE zZQK3`aL;*gxJi9qOIhms>OVAfsOGbu@C&%_w6i`BzB8_OHC!4um6oTA&wh&Ni@t+z z?0nOSI$jQP)xX7*;nrB$dVxL%-6cHrliGK{@~!BhQkKu6L`Mx=KjRH%1rXkmUG-Cn z=+)i-kfm3vuQ#%FIy?8hr#QXJ9k1yz*`@e^1z-9vz3T2WSRH&;^6m++;&V>WMrLiV zHl5`fwEVkbCV8LA!yh^h{XLLtaf4?1XdiypPp;j)8}i^CvbOC6s|CGmvi&|I;YeOv z9=t3c${|Z#KYp(D=%~d%<;K(50e+56H@=GXY*Wm-FV{9pet#aj#`Mp!FFo>~I1K-2 zYd_)lqCedhKWLxN*E(H__x~BRkM;ty7Jt>re?va}n7uyqx11Wlt6_fb?YCpWwSQYa ze5RSmVovEc^{`GW>(C{EGtqq_fY8&BV?=3rD_- z;cE>}lf$_*z)3jkOU^8QcJn*7 zt;^Y6|NobFnP2Nr{$^coH~A}Uonra5B(eDuY~Hv3&f2_Ia57v2*BPw9E?j@1)6?8~ zVXK@`^+Q7OTYo<22XitT^Wp$AuKS1BnijB@by>V&J}Y_YJa{o%nzQZmZHxT1r!Ai5 zS0C@L-*9!#V_Oh3Zb}kui{$MGb9&IU>4<^n>`#CnuRp3E4xL`TZ`qQvSsK3U3KlrN_ztiET1N?aXvtmH1?>zk5e`4CN zTf&jNUq0FOPn`dd$)pM1B5XkITMb@di|+{Va&=rk+89-GFoe?r&R90ijSn10{@UGa zoGTwVj=s;+YVF1n@+o;2AB5AASg)!g{tt`ASvN5F0S>~!hnUblz2J0MyENx;?yqtE zme9`0^2j&7WN}s;&KD2DVSdY!M|!jmPW3xXzX!3zj|Dh+^c+87T+e+8gs1P7W^OQe z+fJ@)55vnDH$G>AU4AZY7W*IHleVl8H-;_<$~s^_v6Hx;(72afJ~d+G*LqsXO9zMvrZQUcuNUqpeOaesUjN%fMfUmyeYffGHT>( z;Zh)Jtuj$6T{)cWbI^JdQO0g9_-q``(P_CTmHNbf890pA%nqZm*C-ZKD6O5y< zzGJyK`sM0Q=ZvpKrf^nc1Q4Czonc;n-g`G2)%41c>Q!!wQFrg2lIWbhNQefiInYPd8e#@}o5 z^y;BfWN#d9+lU|WN_m!sOpI*Cj(PHz@wI;g%KIdF>Ub`BTv=M*_$hkUpzw4Ue6p{_ z*UHuRJ|m+WvnsQx)O^*sq;M~hPwSfIn+@Nn)zuhZx(^?vJKbOkRC!BWlkYctk+p7E zURsOSIA2norI)I6(JcmRE0kd^G0vQxM;=}|oxEoqZZofUL8m@wa5fLaDM?Rd1IdAS zkc?GR23zCEPx@brNk90O9?XXJv$ycKi)-XB4Nl~2(`MdtEa@SzcpcW4Z-|(rwwvSH zoqq6god&1LHGIl4cdb3*|q})${=J zOQr^!-}P72jxk#pV+5}BO7qsb&9CzO6p=@|81c6Fuf;b8-{9YydUP8uCa?HehR9aN z&(Xr)75HjCZTMy#UzXp~oL5Gdqif*ko#pfU9~(6Pj|Okj;Wg);{VeRy1(4m)S-PRm z82$;zpH=@7F^6lr@UzX|4pkhjXvyHLhd4QUVf=cNQQc6?HGXgT)y}8EJ#BH-etyc9 zkJU*io|yw`8GoDqWaJsm$ES1VA!GR2=2!Ux;8gvy!I3TT6K7DPcE|t8c58O@Ui~r$ zU-M@TUx&}cpKQ7FG^_e$7yd5G-y8Dh+N}fBqz1nFBIaj{qki&JSE#4a@uv4Z zw`_Lg&eM-G@qRm?AD6)GUN-G9=5TvLIW*2XvU0x4wkksH)(8LW=Z(D0dZ>h5b!Jij8?x2@8gL7*rpUi+^A9PnQsTI##SakmlTrHV5gV>pHTBqX{CV{=$Ij4eQD=YwQ2TT#mOWtBB$0J< z*7^c{Zw=QMs?P*`n^vC4`XX_>=-*sNOBeq-{C!_C^*Iy%3J;Jk8~#;3`YBI)=wB;j zm)`;Z=o#`$KPKR>xYLwh$J}PiKk)x%YYUS7@3Fhr)Kr>*w*mnh=j0rz*iY4Nm!{!I_b8 zO3WjLQz!lR75rx2q;pjm2VsVz3_eBvl;q{s=`IK&9RNfy&Gax ze~*Ay_XC61jE(XWv5kB`(;&Qb*|9`G;jM#L^J9b8cvL(uKVCImSh;8LP9GI-DL>u- zc*{RCc&Ve}trX?0?+SSRzc6_E-J&D4-)w%oic)N1`hWc2f_t#?bt6m}b7h2ZyI| za{8q2hiT+}>u;O)17SZd9lR&&lv^)nM^=(PO@}1B@H^A^edZ{y;~A#B)2!wE)O2#_ z;PO}jkt2HtSdt#(D0;#R3oE;s!J~BgnWns@u)M6_jZZfMl=H*!;^)^hQzOqY{Pkh` z<*YZee%{=dK>V1a)X?u3{vrPznY8zR>XRh2_s3++OK+oJ-`% zZZRW(aO)q+oX~1;8`(?vsfquOeDE3ig~M>iz%6@mj=WQc;0}XJo@9A7Z)}6R-eGX3 zjI_SJck52AlS|^i>zlP|COdq!Wwhrj49-Aa9M)`y?SH`8=`=VRw|>gr$3zy6>QU*# z9DMa3GV7@!!QB~w zWG=QJX(wNGIrEPzH?((4huEbae2?Twwo8&{2;BM}Q=fj;=zhww?|y20>Ig#7VXW4ech zWN1*?Y`PmGCVyAJnHx1Y5x@UbV*H>aT}Hp~E^G`nh5DoH5$soQG&mJbo|coDCe^0| z$5P5W+beH5Q9?XX)e)cCp&7pFPZ_=?_TqkG|K04`L8~^O*L%44h)%wOzhznjfGRSN zz~A|ghJPm1uQHmAa!f~Qo|q%w#*)d`$6C)%{Dxu2hYCJes7XvtPB{bJZ+Vxb*G`1h z{My$;*SGJFNu9$Pzb_fQqq+@?~&f4ymP%u+JZu|_pPp7rzmgnP9tAe*e*Hir;^@KGu?Ms+YxRP{?2v7FQ22I zSf?EFTf&MNE|r){fV;d>WYo&lEV4yN0gy)ewuRD z?>FU4`@-`lYyA~Ha-}Tl_u5C1?X=-*)JP_d_&|PFEaVt!}drRNAhmPhLOJ>UxI(_ zq~V_p%gdV2yQA|tn?dJnGz_YjS4lk|YUOt0G0*P`>MJAZ45K>gSHhJeoK|oe9%gVx z!}{d$lP$#^hsMnN{Wbwk82dEy>v{e*keliJMeYV>#pob5-Qxg6W@PgNTJ z1-}yf$>Y!U<8o0kY`>^JwU1$q@Mwdx>gtpCf73TJhGPcFFIo0y0G!gt8l1qN+8K|3 zmFOon8biOh?Cu4X3v**fc%pWApZ>c zt1dG5lHX70uk&B@qbctHRH|RrGxUwk-)nM9-(zRIbaufV<5CvYqwLq=wR{o#9mMAt z<|aHxUcdjz-|-$ZpG^xy3Ati%TwDQ8jrCgc55Tu<`DXP{DdBvD|G(R#huJ7U@G2NC zUyc>3I zqJJl}asE(Jf0jLtxKpDkZ_M%c#m_wyGMwFRR9RH7X86`FGko2SuOrldCf&IHlfUIA zqc`IBQ)C@QGsN=Gkbm*H2EWtgZx8jKc4&0{q_T=B&QcOsNK&%V2G zHDTNR|JeKfz`Dol|NATH)ka+eL2z3^5X2NgkhT{=5N!}-nzTuqu4#LnU!Y++$R(D8kp_c`bOc<%c)?6dFp^T+o-?agyv z=Q-y*=Q+>woacG}yw?0}hMv%ctS9DrXMBHIuO{c=PCZ8BrXO;ST9(VkS1ZMgBYL=z z2NLs7Qax*!KPFDJM(@kQNCl#GhQ@>cU>x_ZW&UlV-i&Y7sTB=y=uh%XIpm5qvs`!7 zFSh$HcL>{_*v*&JL-mKDr?Y|elt=52^9${%)hon8awCu%{G4(nkn@9xwYk!VLwYO4+%I|THRw7YP7qLCete4Md5%)?m3P^{IoKvSrV+m-Y*tAh&6A)! z4-ry*2hOCA-fa*&sClVC(GxeqAzZ*Y_G2k`ts#jpL(_L*=g;Ec>Ch~WUp@=`of}btLnQciN4ZUzgsgpwi~QH zaKS9>N8}lvNI$HI=u7Me=!~7fcE6KGer$!_i92|Fgw)QxB6{Q79T%?>+fCzT6#DYz zd7UBE_v#1g$B^;+BU|Vmmgrd`79hKFy|{;6^t7m+-$eAp+eHU;QI91UM)RNn`g+^B zo#U$S>SX$0R#czm|6a(~-OchND!(QopXmSC_mtQ@L=PR*P9N^&8xFJHO4WN{L~ndM zW44#LVi0E|iITob=$pKk_0_4qm)=+Z$Lot9cX5k5iBf#0dltLTW%0rcR_MJmqHo80 zKzFwnX9VJQ*K`Sw;*b~j_)T5Hdg*>*O5STCdPiqCwyVYl4Lq^?_UOeyvbz@gGpbmB zJFX*8@}3>hKZD)D-R$fpz5QhO1MFW~%P4s_epbC<+-jBR51KbwI9uHQQ`Xy|dJm82 zO&njiBppA)aV;BqDxqhzm-Q53FGk7xulKb7ZI9vjHB#vQ*ex%)(|U!uN#uIyp2c2$ z)Cm&<3=a<|bz2@XS`@dgL8|vx)mEpG!rv6jEVW(~HHwW$kZ0+f4V=wc_TGr1$EP{(M}Y z4Zg1G3DVyR{roe;woJoHv?ssYVi0vOy{ZGBC{-4SI1vBaIhJKHeuc1cz_nL_Q znf%|fBmYx8_4%;>&1U_%ICi4sJtm?*X`G(z?!XcdA4~Zto!6~e=X-jm zZ$I=l`dM!z9z7M&JHx#sT{u6JI2dW1WZ~Yv=6S3)OZ{*|L~rX1TYF4C%|o zy?)K2uTAy+c~bi?USGow)$uhO=o=P&{i^TwB>L#$Qgr{9U>}X!LAn<)pWB)%BY zeTnNe-Jct`A&-Y-5{&~t?(wTXne_&7tWC*#Kt%6M<6w(*zfO{VBD+J--?onRH)UJu z9o|3wQ~jEBUU0Z;oomiH-Or6cuj41ImyVe!c|VTm#r4}A#?c~djKE_*zQ`{HnK&=h z$NK3#U6j0!Mf6Y3uwTrCU%+FppT=7o^ar0~{T}VlTO#^x_f50uebceayaXV!cO3e1 zo@ad>I-i$C^o3^dM>pm(cVG{CFBJWeb9u<*KJO}Oy!>5sRCzeO4e+qE#VBa?O zCq17{$@}r!I&MSu_e#=TP$-JJ(b883S-NL2EcdV#sQ-Q!(T8gSNyh7J>rHJak1D16 zP3YWe0@1RP*Q05&|7EtnMD5=ksc%QWN93?#yHOvhZp5b)_Xu{s!g`BzUK|zC+rQ(u zUX9(0n{Pe?XKahC?Y}!0)6%9u+C}4h81)BV=la9it~cJ&eo1og+XTW$T(8h8MB^#Y z9l$3)3+-*{(Qdph5B1v={@P4&>@!@YbH|71NL)${3u`Y*|Q zoN+$dc#E_3++qR^$8-h_Ra)oin^AA=dtC1X5s|$2MCvu>HRfc@yr%g$2>C5?uj7a% z4f%}``6TBp+syNpmL%!RqI(17p2o21J7y+*7&YW9}80~Bo*X`;%w}{c4K=Z#|9MSYopU03adI{$j#)P}fyAzp z-45#+fgay_cK?{_dG<~154(Fwn#_Ai$p3!aBRIS_`>jaZc~3---Mq!YMEty606n$P z(=PW+_NhJVBYG09of8Ymg(2scdn6-rXUr@&0=W*6>sPxE5IM82#U859NB0sQ!0nC9 z=UF0W*Qbyv!>1T>ZGM(((((1ixcYmBef$K5MEtQcjR|Y~LvQhctXJQW;r&%aubuzf zP5&3dzH#Wul6yV7fhl=!i0BC>469&tRONv;*N=7{#C8mjFp~G8hH|Mi_hWb%QG#1jNrufwfeHEv$zIOHJ zs}Xzc-)%PAU613exEHXIKI)%o=<8g_`s#Fk{34>y7!O5Qs-ok8=1IYP#J@TtzC`5{ z$0cNFp3psu{foHXt?HM1BK3~$V4vR(pQAkL8+~m*&67dsEj*3&_5xG#UK7z9w}0Yy zHzi)DC%@1=i`l2Mo>of>jq`|}c)#GZM2uT3zm`J2UgR^?ug65>lqmXE9#&qm}!_Hl;}@=>Hsl((#qJ#^3F zpy-=cd+voi{QQ!#jphT#B zoh_sg$@^5K9};yUro`$be$vy2Ps=Z)KZ%Isy=ez}Z0-}wgAASP91}h25AUjop2)Lu zq~Ff}H0MZe`eC+3kD2H`DZa*;Y`yR)#o=pE2t zMzXV^2z0Vxvwax-Wze zcdXAkmyxKD`m7h9o~Kz~KE7Kiwhyrl(>^5raeO{2KlL50aiz8F&JeJbY{r@0J4|TI z(7Ee+v5)4lmCRU$Qy|gHPkdZT_vwZ{r(6z{4??a*NG3I3SvDxyd81N8Va*+F{> z^}{rCp`kYK(v3XET|CD{n+PwMP9y_vwy^ zo&r(hD2c{~a_vTxWl4F=oOl^+wc@zWiHzT{O#{N~{ng8nIgvQ;5Oj@m$RtMnUx!bV z=;|gSlK0w(uK4{@{Qc6Fd>8nqM1GWnk-STj$p_;GLK%1`@Yy1VqzdpDoan^%(2Pl$h?lKcc69 zhV`HJzTi=%@$0|Ue&`<>VEtRP{hK5DyLWKjkFKMjJXSY5pagaYV5?W+T2S>K717(D zWFI?hbGptt?xbr1TL#(AHr4m(OX|(%({{Y6Bt z&G7^_?QusFJdPJ&kGf6tv_mf??==xUanD-PwXMqN;kY%=>mgqtaivz?m5%%!9E6w<>J9h~VF?M*}o6c<&yvlluJQ2M+_Mdf{ zEAf2iY*CS%^Uvy9FYO|H2 z_pmg)$#QK`xwzwjxMw|3Ya#dokX!Kg1i6I%vfE2iJDVWiBJ$*0E5+G6WovIA=-(vg)0(gmj60h{Z-0oTSe~rI z-zn(Le3$L)jN18G_0sE>V)ep)+Jwf}-3$s6?e(m+FY70$x(>iev> z1K3Ko?SjX262`Qv5%T>aulrbgzhIAU$Wy!eAfNvM*Iyis8#A?QdPnV|`JCy+G3w%f zCYAyP75MZ)Z>4*?^{${Y#55}f=8oF$6Wy&?Xiyg2)1fN*1v-PqYN&s+agC@fo%P4= z6XLI%;g$u8L!?PVBL)fiu>$hdvsm5}^+syhxR2}0 z$9F3Q<|f(KyUVQmqT~9D*Y_UC4)4QytRO>*jCYg7^oCVU9{x^H{UTRvQSqlVdX9{u zvc3bt>VK@4dHafgqW%uW?Juo|tU=;orQyN$92x1Pef3B->kQ+&m0)|qm~hd!iWw4M zT!2qGKHhRZgo%k6rSW4D@j_1uz#^)*6`zpEh49@aH>(3w4u9nz$oxG~{)9Qt=OD00$HaFP$4>Jc8E*T%X-g{qtMvpJ;&+}O? zaW@7v^pMYj_=II_M2_cv5Y?ABMzAk5#t7-@#iw8Ts#n_V-5k-A7%$v(J5GFjptx=I z;Q`!-k?V$wBYL`%pXYGXbz782W367d^^snr*y{&0uF2O^(obX2eu^9;S=SG&m+?o> z%26Mb;uAiIZS-XTP@3sj&fSem9Eryqq%Vw5<oRiGV7-nW6qJGF0!*3@~vlbeR}T>?I~m9sJogDt;K61*AKa&5|+z| zwl7Z3H4!If?FFVG7e0sO%JJPww5Qt9F^F~TC1DF{=_|&1-FY7C3yC`0y`!1Hhy zPvYVbojx$@AAwx6$o)_1_ajC$mvQ~nFIF<@$I(P=_11VOhg`eJ*_q5Q97R zpe;I)$)BCjSAP-PQ5PMrA)En--cwBrFQ0^RKpD!S4yBC{>7n0CYP*>AWJc{8PNHWB zdQg_ArvUrsmI~H09_?S-HP%hP5)gYSjlF0+n;U|jyvtZmPt=|rUVFE;&*nvUimG8) zn@G3%gWf&RP{s8$O8vI;lQumXnd#UNV>=32umTyqd!R$~#k_xO2mMR04mbOk?zk{% zd@M!3msDKM_R{quEA8<8VS{M9vB2Y{)QLnscA18L|4P6w;cfi)cG*63h z?AKV!`e;3{k}*%Jjd?kmA@5tw^~Ri&+~N44t3Kv1 zp{mIqdiTJ9=%c-im2CFqIG!ZeKQtE*#ukz+2SpP(Two|Qo^7kjT3k)!jYR-*S~?|8s?aUczzX#9x1cg5;| z=p9F`^nb!}i3^8NMUlfVW+Zuf_rQW6%TK6$%tVGAorD?^^v@>04XSvhP_N0}hrJa+W zcJ8dS^So}ytho-lH7Nb8^t01T($7ghH~l zL2{&T1p4}~XMNSt{tEB-nLvaiaG1SCezxvg`WEXOqe>{*^;2JBKjndyelMv*R{-I7302OU3Al0Yv> z!heR)pPqwF9ny`YodO*-kP zcANAkg#J^}vz(M5&BgN-)MAsqFX;HGx&-=Hp@;Z0x3WDe@SRd)hCTgy&`FQUzaPZy z^w94&Q@c$5qd=c$LoWb5*M`0j^e@=ZSD{k7dV7-C|Jx+?KLz^1Hu?`1Jy*-P!Oxpm zscmPf70*psf9csR#x!93zz~M;U!cTLMlBw@WDMzGPazggM6<41y`r)sF|F`36 z!CW5i8dsMI*70KYPb#kRkV?f>_JhXNjWTYM#no0R|EzJfQ}~m`)kmcq;l)a3T+_Jv zrtl?;t261*xH@MSo+lJzC><$S=ZRT=Dz5&&*{}ZC=!f45{@;$PetH=LlE&3D1nYQ7 z#Z~+LUMj9y!^WX;^->u($>M5*lz-N^+A93X;_45je5SbijPNCkt261*xcV>Aqj7c5 zU3uQQX zwkAamKLco`U3OzDwEvG5?3Z#=&x@dwtVthFLjMSKvc%+fqj^NXeunxU0w2v8Q@#&$ zYL7|(4d`?po3~nHP6U{-EUjlK)Zi@c9^ENTFzKg( zeyk1uS)fxN+40jj$=-wgc>%u1CXNm~2Z=r)@upwe)ga8}$VtOg4Yxa2=qev*V0yCh z{>@yjd?RK2qMKf!*RJF8e94O>uasP>i@ZBm>z#On<&u?WT*>9iS9$^0JN_%Cw;J^- zUF*$V$$Z5|d&i``$;t<%T=@p2-XWoPNWFf^izTm?yh!S;Oj2*MaMXU$ZH&4B$#3w zB~y-Uq`FMHH;LbfZs)f-N%)Cw$3Hg-Khf>@eM#uMgPv_;{~k%``y|nmE%a^~Z!4jb z($_&J|Fno7>6#EFlka%pGwHS9rFG7vuLh0UY0}%1@ZTl;pNbv2-tRTS^Fz-I{{wtf zR<{?AAo2~RX2E_7pgp}crn^Zdx@8YPxq%gXK{VBhoLkq zcKR4Pilm(Cpmd?&Jcf?138uXTr5?d_&4khug8de9I9@m8O*{9(dP^4Vwl~|QdRGY6 zdQG`2)49KHGUQ$ntnD-9{t7v2n@N8Gbn1_ceb^4Yu3_?R10Qh|3STb1Q@TR%!3-1h zkI1-f7e1Zm!fx%|={t`S=zkajNVPxby!)kNJRI-0iG4ITC~5hm><5?{rEB?icl>&zbj^#6`nmPiYtk*11I9i@ zexL(M-~ILn8HbxKs>AV3 zoT{hcfI48G2MkBEI2ieb4gaE)IQOz$?Qc4Y3y{?YiY@!sS=8s{e{wte377F%E3 zar=vm`4aJq*jYV8dG^iR{z++nue4vpKX?2H%}`&K5r1@_srj@~Znj5{tIYFX0UY#G z+%xH4OhW%I=(OIO{A)p{^AIL|Gw6DZOujM4(SIwh4#@mB7k$@I(rG_DTq zykq_Uui|Qkm+dgy|5@Yaxd-rk)bky;$h=+0ZgbFmLo`QogfG|6{CSceF7%@%KTh&| zDL+B*0>N+W&+_j}{-NYWQclMrlw8}m$MpQhNoXd9l;g`nPj(*D62>2m0$aN|(zlBK zM}^@7(bFLKV&rs<#jKYOKdt!tw|IL7!8;G1ph5o&Xf#j$A@tSwPKox>y2mDLj>ksX zCr}wBvs~x#1ut@m?r;6zrG1S_zX&uM6ORi23Vf&Z6F8CL>On8X>(TowEGGOh=WF)5 z`0;L*S6{^C%2%{Kzh& z6OwDa-NLW+N_7sue9?N_#O`F}Qx|f*%9p>2+nb|uOXAzBbhBQqPxJ6qET{FF^LkkP ztmV26XkH%qArIzIy_(mHKU7}nnhzTFD_!$0DNnX8?>(RA_iU-)K*2OdDFp@V_&HuM zd(YumB$(zFrJIwC)0>1(>-mmg^&X~MT>7q*>%Q*?f|ak+z$&cm)_k%~YhVtgYp(uz zO!N&O#Qyk$;BAt>E4lkXF5gY^QQ^-Ke7NMtNnRrPrIN3gd{E@B5d2lizb*NXCGVB| z_mclb@^Q&OmV8q5ISyv~cFEy>WlNvHE6M6al4|NDhs{jT#rS-BY}M!&=(E`Jm2Yn0qCxyFeC!BU;WtXJoMvhuB0 zuzuw$UlSiEI*fWXPMG!Te9`u5oYi{G^1x=USNVpeUX8o8M!ibcdOO6==MN+R%@*astg@Rk8-r^+nCM!4V?KkQzo1xzN8QPny+^n}?ZG3xM zXQ;PM)`>yk^U8P`zJU9$OmO>B#^b`5DRLu%w@W@Hd|N~=V8{)KT(9s?Nng^s`l9hkf_3CknGv>bW)R(!Ypj6O`oQ`LF zhj6^pS+DQ0|9N{WQ12_Bv)mzf5IU3th*>E7)k|fIdwR2q2l`t@v7|`@Y8cxbVFx zdd&HecfGxy)*F~!F6FA{-?9%>J9k01%(X0u|D){+h`g5i<%`azikrEfjuy`AZsh#Z zG3NN9@V^d!P(2NYvqx3`)q?5RkkZ2f$`(0nYCey)%ug zkA+YDpyNH+d2atv@qSZ&$2j{>&!g*k@j~I(g?xS9R;_~l2x;V{v;N$k;e z?*N=A*&Aig3SH;#y`WM2FzI)KMj8u#&-%^z)UgQ@?lY7(i@a%n-Di|fcD?S{f3T-k zT2?q5O~}dKcA@J!{AC6n)Wp!xCntMJXo zcSH2=LGvz_ZOw$_TzJ{(3e}%e6F0^Zi3_uza#{xZ*9=sP?UgCRz{P6#M3jW|MCX=;Tk6elKXJ;xp$M_VYY^r)2Uy z02;L~Pxy5HnS9>?57l#_@X>clCf~0>BYovl+}>Wv^WS6KDA<&9!C}-NCViIBpAi4l zf`?KAJ`~?gI@xJQ?*pAYVe-@bBD!+{`-%TL9J2*;bWM~SOrq~q&}koP*3*o3*xB`9 z68g`Q(4R;`e>w?$8|ZfJ{#z3I$4Tfebf2A`E0fT_BJ>xIWsmCd`%!PlkMo@n*U2y6 z7P?+vqjhOd8~QTPa&71rgGO`9)N?avWPwTl9%xkWLC3Lubd8MCse+lBD7TCF$JD3S z<%(yJ|1gRCq3{>^=zNja>yBnUdfl$3*&? zFSt_bJyGz^|75+Z1h0qPzzr@0G*CEOgh~^Yez2vJ!T+qp%rk=e( zr?FZWI={(O&`a41|xQO+a3O{Sbdq$+*f&#Xi<~*hA1fRmt@t9ye9{sam z3{xwO80C(4xrXIZPU{T$-;|pWIlu6kaui?e< znsj|Q6tul<=rng|o|yb(2VH+K=?8;GKB>Q*+c_?IeH-I;!4FCOLGV)2`xOtKV!PHd z*+G7shtHZ5+0%Cj-Q-&WI<;$yA%6jA6xUquv0jbiD`ekd(!T{d#nZ0{Klzo?jS?TX z{D|A3<3RgMk0WNf5AynxxE&P3D7_-}=z4k^>Y+Gb((eJC#-&N$8F7;4?w>?2^$R7F zkDi;On$LZoE!OdHB50OHg>09`kw=6tgmOxi_~g(RN!o7Bn`Ri#U2?o<_S>P*cPu`q ze!!Zp68)XXY3|pET?>S6@=b$I{ch6fxFgqwPS0hWU_+<%o8q#mhxlpVVA69yr*+Y! z)A~*8fk~&wrH-(n-=2i7&)=DPh)!e2q|QF8c;^55+B<4w7draq+_OeoM?Q|@sn{fk6(&mluSMv zW5i+7zW^G=xPwIgQGBO#uTc-_B|A(yt+Uh)lfEBl6sJu(`RFKocKMLoq49{G%cVL@ zdM4-;zji)_+efjO(l?K|q@c=_i+r+KtoDN$?SPs%4z&kTD_h9dO1VKWyq=B z{za@;^;{^rmGn?ZYj0YNPCcO!Cit{G@x+L^(fll+r6-h07vDQd z*E}HpJJni6cwHj>_oAINcTD;!%xfAqVd1}&n33qRAGPfoX{YYDOuo}$4z<&ySBktz zKWi3`FO&W|H1|LoJ+q-d$A%193v^nuP5ND+)4Vh3J)qM(FzE}?u7hmoCnllO@eQq|rk;u<^as&i;%_tB zOOHR{e}W!*&%~jiYtnh5c?O;6h)GJ+V-zpC4spdu&=;%6v*UaqBs3H|^^vsl8dxz;Q_JXL5Pr*^)~7Nw+IpVkM8Ti|V*`E4h)?Xt0W*lFQ;vCpzj&FTFqgeb#bAS@<|L8c#q!)ok zdQ3XSFN#wpo%RWKdRBry&xW7&v9zw4dV-+Q{4wdY&zX-;(6Iku&}jUa^iFg*%{P-y z_Zd*%ne=BsqjlV*9|Qg)Z0K};md2;aUk*C0OD26C=<{sobe^8#h{=Bn=yvV;2y{FB z{}w%M;ve0oEeDH03kQHwbS1LI`yNe=e{KLZzj=0 z{Yb|xrk(-N=~%2@=6x={QyLJz_$`F{43JY~2n!$WpDDd2eDfGOoX|&np9tTRLVxyr zqGFs1p5bRH!~CW7(xlUMH)_jS!mswxxTSfd z$&5>NB~XYId>9Il>O`JmKcy1bNirt=8=#X-)e;~5_)baVU9;#Ndx!O|UCQlLy);H+ z6I&5|zwwmpuapPXSxNKp-LWtt6()(dqY`H=Cj2qucZaMuTCR3$UMcl!eacsGA-7|W z5EctozGA_24vx};;!hfjl=S?$NzX@f{3yR$&?!cl zdTs@sjv-C@jY;TtCeia)68^p<^q-0zRBa_4Cz=Ojya)cy<9$s0d!X>&iTn%rcs)FR z)bBsWz#+dMdlA$1xSZA|O1>}g%oZeb@OV#a7&+?`l`A@Oj^5yFQhAG$!m; z&h_$a!dy4-C&u`OxpC=8;iq+<(k8*QR#N(gU_CB0%iE-!#wsPh_($VZH7X+^8lP{;k%A@*y0oT+^i(*6u- zzsdh=(MNj}N|S<*VCdNI687w|f|m%^crIi|k$ll{)Fkt}@L%!cL(4VK7QSt}ayw6u z@w`CvtwKl9K1usa$c}gA{E7CfS>7skA`Dr{JWi>Tc%$R#F({Y^&Yx9qd+8XG(mJtc zK10V%$Z77G^e|`?BTV`aK%=?+JJEXqzEiqc^q$Dju^joX`0V#Zu3yjl4GABObxME3 zhw3j8zC($KWb*w5G~)YM zKVIzAae9x~N5?{x)?gw~ODcs=>5mBCffg6{HO<&=;nVZnuLvLQF(}dAgyu-@GWL&- zqq*sfwLgy#{~jy!ToOha$A{`P>FY5Y*SUGOOk9hZqd8n=|RzgvZ_@q_w} zV%2jZuk+93qrExp0S~K;Z_neRPy28!cxXI3W-*`ADMpbFlTJFxhLeO}{rN-qg?za| z=#h0AG@9!_5&A-Wr*siMG{;Q(iX`-Jflhtge;V8Q4LP2;Q}SVYlJ6<`q}b;de7xkRNxoF_ zO38ayu%3FsH%aa|o#}T8{+Z;1l0Pf?o07jTxl`9q$@@jWU-UgA>&z>XzazQhQm$_| z$q$tLD9Q6hzCiF|$!K)k{|E9-5zFE1L3eKLI*!#YTjB+@$OE z3X}fRB>eyF>k!G>yU5GqQ{(pAu$|(CN!R#o(!=lr#Uqp60XoGylYR&2v zHHs6~yXwbzU$|E6)p5}YURtNj^R@r!^*E|KHo+__RX@q~$ClA|Ef4H7%Pg|6mN$$2 z%A|D78w~w4rYUJ&V91-tYhzhlzwZgQb3}5}FM1x^q(2QqXpi@Z_?N~uB|1K(bi~2z5w(yK%cvk zoqbQkrE1;LgjY+595upAv>H8y&Q`|J^2ZBy@nsj=e zllsY|(_{8zlS$u-_3Q{6`tQ*|qG#>I?WO*qv>YG0Ze-F|gGT*o(&@Z9*}t@g^^{9q zE%|!M!{XO3i~lMYGJl2SVWD3o`CXDjwUurZ+$?#UQT{c--g$rc!{ArF*9iS4 z$>)i^_Zj@x3BFbG?@2x)`t>|OK>VreBJ~xGV>7{RRvMon{%@D^Vad(-?KzO+ccbL} z!gtha_OFi9^ecJ$twN_hr!)@3X#dkCbn075^ge3(ZqoMzo#MYqzX0?(pg$#g z)b4L!oYH(B6FSWmN_su?Q=wCgrF1RyP^`IqHMb|zqBk?5;61turxdJrW2blEDN$3qp=vOD9{}yz+dOrr8;+R=4J;%434gD(6?CiWU37zhz zwd1GfVeIHTC!ucuo%XS&otr@?ADVQ!Pt4B#m7v?Tm#%lsx6wn-iP_mXSLkb`A9cU+ zW$4%gACvw=&`6I-|4|b9{YmIQPD1}l5;{GnM03xqm;6lQ!K4=@p+AsBe^(MZJ+Elj z-oGWG)BPoO{KsIP+Sz{`=yvwgYxL~suYykdAG5vxOhO+L`W@202SS$8ckrROVbXsA z8rg5sdy~)~PC`!?KbU&xK6X3(SHVsi&nS*dFJD1kz-b(zjte&Vd{VCa^xU7S1Q+Q3 zL-X*Dxcm|E3+**2y^IgpXVTvk`i9MH5BZK#NU+~R4o6rp&Dq$bc4=N<*r#;OwcPh5 zu20*ya|2_I(>a2fO4r-s`TYhy!r(XMO}`chKe{I_>H4Gg4xi8d%q`=5tGowdoA_7P z!(&lQ`!F7?Yy*CQK z?uYJ2L#Qp)U*&e|e&x%8PY}Jmf;Eo4D0seCY>O9HoB>%SLy^_0Ro_Qosmz=ex{{FW-Fuj~-fM*5gKFPBs z&y{?EqST=axgUn);|AmxFS2T~qLc_8J1lm}8C z_`l-;dhW(9(eq88C(-#yyOeK71TCC|9!f&*OhO+_LLW&&pH4y_PeSiD=z1Q>ASc~V zC*wEe?QwExK6NiDNO>UTfs_YQ9!Pm0<$;t3QXWWoAmxFS2T~qLc_8J1lm}8C zNO>UTfs_YQ9!Pm0<$;t3QXWWoAmxFS2T~qLc_8J1lm}8CNO>UTfs_YQ9!Pm0<$;t3 zQXWWoAmxFS2T~qLc_8J1|A_~7o#zO8LutOCH=6Bbp`d@sLZSwJAvgY54C%gL?vh2p ztpEE0T#r?E$b-s5ZnhwUtO4M^7cC0GBtJ~@Em>5)IGAqWKn1* zlJrRnb|zg`b1i)q)fd`H8puq-tPK98i-NhLYgU%!6dx1^EVq;`B+LB3i-SG}xgmGf z(#0VU)s3Zm-#V&$gBHrHzM zek=u5UeXK;m%;$=(uG04T_!FpgUYhgf?3F|&L&IHU@tkAtB+}xn!0qM`j866o#+Bf zMQ){U;nLHVEGk>bZ3nXlW&Dr(F{0Y4f>18mAqY*kRB@k0crSoM2zWzQCtCJt#|rC3 zi&bIh5K9i$p>F(%bTIcCjSetC;RXC9-G>g8QO3%&XS|^dFi9h$c8PzWPMf5LqN)pI z=8&}*!&E!}0rzY;7>%WQWL0E|;}5V71#c)jc4YWjP1z#;7xHUOlt&sS9S5CD(Ve6X z|77lpKxnSjxm4L=3=0y*RE}u4XmL44Ci@D)7c9D9VQ7{Y(`)h40CY)R+FVP_a*O2( zYcMZej82cSj(LewE%5p(p*0hBNc(d!)XNry_NRJ7b9|RBT)OZA3>_XkG>Af8Yos1Y zvkb#CP+tlzYJ+49xvs%thlF88)NHaH-$c=3{| zb=B+Fo>8-|WDG1cm)zsIV*07=` zSigGxIx5bJDL$j7e(~uGiRzE12CC|-mekZOs@iC&%#ANSvy?>gP2!pBR<5Ulg7|`x zs&%V2RjsO78mw8ddgbaBq?_1^;@Otetb&D0>#H`@Q)zL0X<1D}eT+e+F{MkYR&R`N zVtH(7&Bje@twvPFP?y%!pRt6hsf{Oc%YsB)RK+x);RWntaqrjVP^`i-j_maT=a>)=DGp_$2>>3gg7 zy}lM5P>u0YzhPN*4Z_TNt37QAg{#-ecuFWi8{y^^Yc{UF%4&OOEXRuVh&RhOtgotG zQMJ)(RChvgOufAch0*TsPbj1Qs@C2gOeofp;kc5>fE$S=SJ$8kwd9x4STfCEt8vy8 zTxWIDxGe*sXDwlv52Oq)rE;g^&76# z(ze9XP3yEUoKR>Eflhl4HLKgc7=xky%Ahq4dSf{@*3_+Zz`OnN#CnQ+;_G#6~rj1!8}{NcKtepy^U+C)~#C> zK)!I@x^t`QS6@-Hj8^0Z$CLClyvH&0oCklqU!lL9{q%SEn`!vl^8{WK=&ISU8gmoV z<{WF{E%i9spC+CGyqntNn0f|0?!{JgUlYJsU$>DK<;%~hso$`AMH#tyv&Yf*D)A28 zKw6tV@i>=~zVA65uBr_f(04ht3jAD3`oik!i&odyo>{%oanhnqtKq?gi+Mhr5x_R# z;xiDgzPz-?@pCeMeAmM2>9h$~upqC=IpS(Rpu;`j9`%I0&2@dQlEP{!VaoWviL`Jw zX!+C5{4ru+&xa~5S~)s<#WL+%Opl)Gp_ zR^@bBlXKeTX`{kXXPM9E&GL9Mk+wKH-JNcqui07d&B!S7mKXQAwmG*uy-ldyla-4` zwL6DrkGpzy>6xb<_KtU>3X)92gftm{I> z0ct(BeZtj+_IE)vnv?@^}DjSIftEN&Ze|BS0D0D=YVV6o#*xXD#*=k&a&;!zFApFb?6!Hn9NKczGt9M zii#|)85!H)l#J|bpV#Zn$|`P~h1~1)6lP|5eZeNDw*n5#>UD+CV6U&m<;ltoODgNror5S0lU?`-I>$k?sFF`7|&=)YjSmYLe4F&ZfALM z2rdt~8k(G4?m!sTHaojqTU@Qq>f-95tgt)e4(&1N?N8t03VHgSTQOogFq9g`UBf$} z5iM!G&c@>E#(qrEF6T5pzBbpeb0TfplhfkNDy+;XZ*o>w_hYbba}ByWU7c=pv@5p( zYTd;=;Cq|}1(kuiaaR-UVQU&o$egf~+|cZ-XmhussJa`j9Y}BUOd|^PKwon@{Nm0n zX~EzbbWLJvl{b`^ZF6e4P}pdZS*F@oXiv_8+)*%>`MP3+<+ zLGv-SCtMBHRO2M-9I)!_b{2Lyi^FN#yn#_~3;fzNtJl@<_GfN)Vj}f9`@q%V?r?WG zw|R0a%O{-;opAC1hVGCPCX6~Wx4SY2Tn&THDd&QcKxt_~16nwj*;i-0Y?}3+B^fEjEt6a&maasV8R*5sc3W6p>JF*Y3Kr)X}+)n{c5!{<0iM9T1FbU1T!vpn6-Jj90FF=rXgx$+@r zLs@mh0PLK0R?t`uIoqAVHs^v7XMM?5XA2sdi8)YSmsOTE2CJ&uFl#c0P*5DglqkyV zh5hheFsIyCnK$OF^A4jwhVZY)H3Vl3BUpsoy=ftD$4;HjEf@(?&VJVz{`Fh`250xr zo}Lq$J+@P5c8{;sH#B#2=aHRze4XwI=cp^>nw~v9%j@f(<)hi!iC{E_I6vj?$ERy{ z*tuXB@uA<@(CsQ2Kt6ym!hh)-Q838TCex>U2+yE1(26c8 z4wUwKwz`H~{ob&<4a2(2IqYe1d$y!Cd1#gL!apwW1ROeo*x&0M!n~z_?XG5bTY879 zKYcVkx82j>&dqIhdrRu7OM`>1U|FXpr^hws$(VAsyMz6%I?q;kokmWbwZ9FM zp`th=!#99Y(>7}ilcp)X&E4Y3?Qsq6*5hfPHQ~zkc(!7023wupEiOb?S4PHkdLKs8 z5N1Jnxz{`CE^l&UnEHp@eyr4O2o=SBMXj@@T;cP!x;wMBx+u72WO#GDIS682ZwA&a zq%3>`m5O|b?+q=8P|mD=PmilDtMh{QN+m zbQ-?!6l8?4a*n%-`_g=Q+1|{0M3c&4#3Em2$xwP`>z@7ICRafZMpzpHfV0O_IpLhZ zc%61{akXPf?m~PUa(B?o@pNYN@3b|2*wsW!Kzes(X>neUD>IPe%gW3QBh2NGyL#Py zm{%Aj^*Lki)~umDM(0eVPq~K>il*I}nN4Yv7z85-ianUbA%qX;b(M6cPk6SZ`!X`d z-957!o6>td?P}#5p*$J~K{N+t~+OX{R?K+m;@1F3CVc7Wxy%TBM zoDCsp)PLJxaW}$upKHM7^Nl(&;xn&_|v#YSI$<;Y` zVpgCa;PccMps*XoR6?JvvnDg9-5qI#ldeurX+y7P9N{@wSXkKXSrEt^nq63jT@pAm zu;=M>mF46VmSlIj%W7$sx4QbU@yTdEHYbFAOVOAs5Gcq%ct;56fujR?#g${uj5beT z(p^-Pmz5WCE)#^y+IzUXU+_)%0SwX~*oF1oyPa{nk8z;JWtA8B84u${)5O1R9^5XH+wGr-^>S+Ww$mwSP*m+F!Pc<<`9)?YGy>id^yaF!@r}Chbw>-uCwN4O`mb@w5e`S=%<95N{KZEfI z{U8Ie29L~18K=0xqOO#U86u z{Y2@sLs?(`u8d#(6PJ$*{ixjbhNmjCz^)?ZCO?1OaUm$V+rH+&$9l)rHnN`_6UCJ zRUS`Wh?A7IY5j<+lzjN%9rDlQ#f)`+^`KppbUsZ8zHvU=8@!au|MDZzFZlY!JYHMq zhq;iB7hHN7)GC(8piKj#PU;u z&$)ribLj_gkoMe-<%(k6yu}(jO)3+1|{XxII&t7nE9W zXWULdWQ4R}A>%H=pB&5j`UG#kg>hgb^Is+9?SiwV{tm&DW2~=R@cj1~_X)n>Mb@`n z@Xd`}?x|<}D=-hJJr#l%-O9K_@Ui$|2r3`l#QYDODE3~#_>ph3|9a^MnUHqBnElr; z_=c0XJkY@9m4D>&LBYFX-I1>z8uXI#nhd!T>Go(92NFi!~gT*c)X zKVdX<$-|86?_xYI`>WCW7_V6;^&_rOT95Ha?alfL zMpPr>iYex?w1Qd;2W z{@Es&)-U3p5?uNam*@P7`A9NeAf=cJU=`a${}nAemxbTh`e0t>%2 zkM$2c!g$RwTs|Rq$p&s;kbdwG>GDe$m-aKBCG8mz++4p*cfTe6e~HU~urvF&Yn1W&AG7@6%Z!^&=klUg7@s2jQ!MyJ zC;O*b@MGAA(tHdF&P6+@Jp+O-{x{n@D)?EMUz36w{V^)|@ng7rT=4vNSl_hZ%FEb3-)k&?`4TSA6+H1Imrn}*)-Sod_jTsq^d93u z!MZ;j5v=>Yo;R33vY*2~k&^D;h6L+=t@KSUzb(x5ZxLL-ka6GNxt#W6$g80&s4 zcY?9*$BN%#yx?%Q*ZVf(^?zl3_5T!k*{}I08NYQUmxl%a)6M=ZfgdUTO~$9=UB<(w zvcDbwV!ZJT=3gKvS8@H@1)qlgv-Eqo{AkQ0!i|F06f-VK{rPCZM$&! zzPqyi1-mjnshROK;-)2!ekA_D9*kF=!}1k-GCp{i_1Esj_=sB>x1b(Mi|&y61%D|1 zcHlUi(oI-TXucK;*8O(DK3uN*?fQKguRDqLO$grQ8OFKUT(0}gkYL?kHej8gG;}bJ zk4Zmc-Jh4vW32o0OvGhMy5AfT{PjKAAHEze*ZpUw;Dt}KJ$VOlx$fUb1?&Da`(Q5D z{pohW<$q@Vblgel+MhEHB2H1d9`;jzR2;$hp%B*>zR(+i(GM<~o`icq}xBZIo^vR4b>0|z>QyG7; zlyUPy#?R(3f5Rfim1qpLCnLaka9%v#3qMpBTmIo!7`F=ldlC=4r?LD_vfkATPHSZT z^3%Co{U1i0p|tY%TpmKZDP8=1X^-G59^mqkv$(wIuUxM9U-)5Mvahp*%L`s&+=F>S z>G2;jfB(6RKl%cf=aw?Qco)WH=QDnPF5_~+cM7f$e5Bw?!F%Am9o1hgczg@@U#;N# z1P2A{`c*IZ;_vhPYY}|XFBx|TelVZy?-9IQ_NR&uamAN!L0l$#w0!?cwomcV=f~ro z!gze|rEI_Q&;Mh*z7Jq;to}tG#N!`fUmIIK{1=|@Lt_6o&t|+;@QUT!-VwoHkn=ke zg0m1ODLy$ct|&cI&h}*rE{8sjc`2$-@$=Wl<6fM``7nF|;{_{OU-yNK)7LS-;#oM(Cw+={Ylz3U)y3nABje?pu#btApO*cM>ihJr zc>WKY;_*Me7mwHN9k1`!TNw9<{YCf2V?Xv0RKJ#=hJ9EpF2iwCEYANyJpN#u>sR^Q z597;U#t-Pm%Aa&eJibigqsrgi8n18RJjO$6|AXG{H(@#RZ92Y$aq-r zV^|-EzZ~fi1Ke~d;OD<)7)tI-Gj%w%n8nF+i z^r+0gf?CF}KFa0Us~NxTU_2oB=vKzwHC#R)>ps~tS;sj2SZ;6jTE_2V9Vhz~m)FOa zS4+Iq@(uk`zsT=8FP?u)))VF5U*dt{Q?ak1`f^16A)Jqk#RC$rwfu59{#2~z5AsAl za}CQE3f}WXwy#a_M`wt=>$truCfMI$!B^hIxL`e(|L|evZxQ^v`&oZ;kjr)b4%9QI z^_%Rk5PVaR_0-t$Bc;mmgzCy)w zSYMIgAMMWNWrB6Rtre{6ZKL2nW8J0k(kfWj-wwf-FW~m}3f_cyLhT@K@LGXTqJRX_^AByuY)ZSLX``~&J;fX6*-g`6SF#Uix(oLVRe3#&( z|IFosg5Nxh%f|#CzAxh`!7s{suwC%!*ymDxo~v2^UqxSr;7QrvWz zli-ZoxW4RbSl?m4Wt=1U-4B^RPjKB9#tQ^LB+cq<$J@Pve~t5gq;F90_ct>x2(kSye4WRCpWqdja{U8>zb)~||5fI% z?C1K%1@Ez(?U@$*xSY@SeU16+f6DrcuVtJuU+fY5+o z*D?Q?;MF)Dqxd)>Sl5G!yO{sbVJ@#1oO?Xu!n?WrJEwAdEfHLG z8JAZJKB%0_efKbbZ!4Gg34W=J?Zf?!h%OGtgTH0_hXwz3_vp zZkf+ge&$@pm4bK2d?5R31^?p$E)NPmx02i2Aoznf88-@k{T$|R7X0sn7!L^ka5M7{ z3I5X>u21oY)m%O-w%gY6~yv^nHf)~m8tk!zK0%lExS^gqV<^#hr|T=1n|j>q@@fcaac{H#I7l>;pA z_z~jYJn%S|f6dQ$Qt-9kX8q$&aQTUQvi)6yjQ<&C{z1VX&FAqqBKXrZ<_|o{ z{J(yO^%eY{@z=l3@`@WDEjQ2W*uWtjjQ@QAJjD-y~FsJ zbJ?D8!S4-ldD**MzGp4#E1zQACh@*e@Hb@q_X_SQV14=TG5_inj28&r_zdgwywBxt z?9KHB1fR7F^Op)f?@!F%A-J%D%eM>O-^=CQAF#f?<#?oZn(_8*=J$Wd_}+PpTR&p_ zmy<=`$Bf;J8Bc9zT#faS{Mr5q%}aXy~rXFzZX&aV>g+L_DW!TCSJ1wO__ zJMs7~+?DZ7zvJ@S-5Jlmi1m#NK6WdYZ_niNCG)wwYY)cnzs}{JJsEdmJW+img0Jfm zycd^$n#1*bvlu^HD0pwi+wWxlkl>5<6aD*e`Pg3UukL*r{~z|=1xxA@4H9K!UPzDREPHtWCfE9C9Zk%x{X z&;Bmc_tlb@rNt+aJCCEjZWSM~iSpH>DgR5DJZ}v7LHP@ojU^96e_8viIhnksi{-63 zg?#tZ?q-KUX9AIbPN6Ui@K!SdIMJHNpCZmg&Li$12e@N{zI4EAsN8RV<$ z7=N;O)f&cMDSq*IhL4@e@K4o~w~K$J^T_p+DgXIFOmEIt$v=y!{AZEf{;%?E^0-G> zzL5Bu=gH%yQ2vWAv%W*mA>SnXci>d=X>Tz6oD{kJr_67O_~z3X-o~Rczzc!aH92LNULo`Dsjs`Pulh@Q#l>n!Hx>y7$PwEW>wq zkQa*o_9nSLNBM58EYF~665(%Wzz`8}<_bDGF6>wL4WnS7<} zyX7t9XO}bmjpBR5Do-or?KhFvw~>#!n&Io)$!C0yR7m-chExq|-x4x8L zOu1W6R$fBBZyo2CDRanfeHnTw*{v^&#cutmx{UHaV*FZubHztXe}>Mbd{Uh4G3;yP z-EUNTi{C6``_)`Q`NKyt{8sTZ@((%Q`yIx2+$(?PlCLxVh4NqYT}7@viRqVpgWNQf z@wbcXmN37nYbl?zP303G{~|ed9pz{Lja;*k{G$_@e$S2M3fTWPemZX=KXW_7FA`sL zA?1DIJwM6#ojnZSbt2`pH;3Rs8U9Y$`#rajuav&8Sxo+2FY{mh9fki3dC>RBH%=ih6Mt_C>oef{lt0%) zUb&RKXdKHs>`wCSUm!0L+kV3IYlC>iyR47zE{5NI3FQmLU(1mQ44=lg>d&o||2i!;g_Lo_|MZM%}Zf1QJ-cO#O{mJCz8%qVcs0Y%{-fgmoaI?9K1lO@W&`D` zHZ%OHjpX2NY~KOTk*`;JnHR{bZ)fw?!ymgJ z!w-Iq>`Sn|E4GqX9zve-26?g0p9a239w&c)<}LC$vghWD_nyl1w}?$YtvzbrW_UM0 zh29}M|I9M6>5ql4-%h#Hm#Tk~|8@)ITf~(Ik{AAqa+}{Q{?r|0H@__x-?NP6DSwyp z+x|q}DqeFKdBA&=f9-qZIb{R9{qE?!$qU8r$zQi&H_E#;zt!zdu6Tjv8S)A8Dy=Vr z29V!^{cq*15&!%L3_o9d%U_w_I`J!CF6O?@VjgrYDO&tN(|W zlPBtahvT!xu)TIj{%PsQ`U4sMu_Kt?+=IyOe($)0$**g^3>`vlx{~2%A4*<&1k0B> zjC|d}ly4bAHh+-KCtJnu?nPd5IOVqfS$&otK{o%8g`fIK@>x;M=i=9HWB7reqI`<% z-EkG<@*0+BviJtIPe$DHIm+8U$MBaQ%J%6LkNG9jUs*}{PfsNeKAODxD2AUbUMTy# z_87{qRDaGBUoU^sGI9JmmA{JNzc7I9QE@DJ&@U*TFaDGEUyF~UynQ6)y`LvvFqHKf zUQIsvX38gjfxJ@wiFM-nU2Na=;;`)R4dMs=EYC*qX}@B6o5Yzg>+d*m0`uQ0`D(T2 z`T+A!T*mmz!{l$t9-SH^Kd1F-$QQ|Xy}-dkY#?| z;*I}gd4`T-{?SQlAMuY5BsYlkw=5brsY z^2OqwIpp&3jQ?BgzpOqn@qX~X84sO6`LUm7_%`v5vnj8sqx|1P$d#v&r((Zt@n?%$ zrZN7QiIm@j{gTO7iQoDv!w<&iz3ji+f5P;-#fMdrw~DXbo$;5~GyKU|FD$*Kr<1Qc zhVsQ{kmvrA@*U#qS1J6Nl=~+zeEnqdlh;wcUc3tD1(tr>S1G^X5{BO-K5rD|-DgoA z-NE?F#GihhJojwMf4!ExU3}uhOs{qd<&t-Oln=q$s}(`)Nx*mL_k7|I2uVcx*4juN7Ynd(z}R8HWGEdkkNhC0|)aZW6!j zWBOafV;U%5pJVujTBbL68u_R1P+lc|2s@ zBi^Tp^15b*e=WrJsc0d8UHZ61yyS3(U(rhWcf-toOB?y&`^YQKCx`E5`1S4NKWaT% z(m_4~{vj)0tc(1@%ZfjPe1o`c7I|8N;oB~f9QL!tAAA{kpFfgUiZ7`q*IiEe-kQG$ z&Ltl!duE6D-yK|UhhIT?<}R*>i@#33s)jt^O7aaCk!!CaZ>wSaxmS~Kb>n9~`3KVH zLD!S_l)j9+iG1((S>D=jlPh2!Sp90hOFrRz*1zfwGCq#u*^5=n$XD%7dt}OT@&!k- zd`&CKi*TN8_T`F4$xnQN-1BSl&9Wccen)=uP|BA)P5vClkHsJNEcwzP(~oT=KleLw z&vWEQ6D)tv^W@jyPqzA$Zzh*xzid3PkNk;g)W33Z&k*UO*xeW0^f!ig_vN;V-F>;L zS1G^#9L8TNzE1kS?lsD}kj87mDRiYJWO=Po{s@Y|4l4 zMIQY9VqRCp{L2PXzW)iud?M^SE1#1ehV`hBZ+wQlT=AEkM(*5O>A`-t@Q&{-WBRdu zD8C4uXL84HKFsirYnB!B`f7&XqWBx$E#}+^a__!O|Bq3|cYMlV$}@u~e?a$_s`ewl za16`WCr;!1#`J5^{**7o`HFGff#f^j?=fCAn0(t;$)O|3d&u4&`YH12^C^#2kPpLt z#o{mf0{MEaCw(FEuTQ7`&N+enuh$g+OXM5kl&=)mtfYMXiIh)0mg8k|f_%5ecXyIp z_q@_;AYVL>yhWVE{RS&v)pW}LIFk8o5?}a3@|Gsb2mFn^u$Ao2KL^Yv4;;kw28(0Z z4_f+D#gAkEWSqH#;qOV3S6@p0K`YC*PCU=;moB6H0qiF&{=&=2gLJ>Jb{=`@^-OOi zK2&S}4gLyw%{AovWq;4Rj{HmPk1f5`HuNI%Pl)UC{%Gb)D({wNS^e)C<-%EaWAL{R*`^kqNPx%7z z))Q48@l$9Y8-G(CVE7}3k{63}I&bKGkn%OKCoTNIACMzh?~Uukd%mUg#ea`7z0Kk? zW&aOd!T8_*7Tdq=A#&yQjKA{7Y?W zu6>mJi;Kt$A0tnJK3VvRUy?8X8Pls2-|#QWSNw|d_zdaKZ^%dOM}4kYOWu*CeEDz5 z2gsgUDgMP-ln;NL^56GRf0jH!Zn=Q@FB2dBM~3fNNBI>ukca$^yvK6J-zq*<>+yo$ zQ=a}ZkMCLX z;4e`==6Uj#C6qV)iF}&o<7F?B|22o{4|<8bME0EH$xjx`?}a^P{S{L9A@T>-^)deH zx0&Czm&sRSd|UXTuaF;%v%hM^mta3;^0j}X{I{6TjMs}7z&}#Rt6yUJ6_j*d^`D(VG1w) z+5wbr`3L2d;>rIc|7w5Ar-+kc-@ho|x191-;#2k}&)z}#WtzWY|0d6DVE)_RB_BJD zyr^tXZ$6*%I_p>Q3G)5>F}=Cs)GYE6@io_|JqIxSFK!_>?Lj_qJb8_H`Xg-5@;xd4 ztJa&L;yX0|Rf(IBp7l>me1y(_cZe6r9$CE?)4ORZ%Qt0T@)_`dnLgDFBL8@T=9~S< zE8bxGwdLfaVb5898xJHmH?w{E4kjP_ipoEjd^qe$3txMP!f#{v`JW;uo+huVAiu8h zIHi(2TjyD8jv|Nm(0ngmyqxvv6DQ$cwEAoj->&`asw&1$!{27|?qkWVdR@Ls?YJ{Z0U1-y<&-S7X0t?GFX<^tU|3{054z()cJB zkN+9-D?5SVUw@3@CyTGt{Il_kl(+U#zH&JE3D^^s-#T#;_X~`-if-a!CBpZRA?Fs`lf24>zlOY4eC*fB8^l}Z zko&|3sysWy&v#N@zL@QGl*V_Z_`>f}-xi6l9mw%e^&N(PTI=Bg@$1+hn|}JfOZoHg zm)iU>M*O`-^1LOK@4k_|@^c5K=XBqx;ZDj&$v#?g7x}B&PcB$S zp0EAx@O#KL$1}g>;?}px!|$d1(g&Dd*?r`Rr&8W4{>_1uFX*LwtNbO77vOxyo6qm3 z{G~lPzb|}%Jbx48FMp6cbRY7XACQ0YcgCOcBl6F2o@D8*T|pj){ikv4$K=B>-x;s_ z2|18u__CGcq1fM=+$a84obpcb6Z=!XLcB3Z`M{qt{yFlul|4)zj``TqTOmGtI>S4D z82f9Juax|+_ma1XhrqvR^0J>Xy_>INdKIh4V{?ix9?)Yp09B9jmLGv)0I8NW$<&T-_)n<+1szo+&u3*SLZ7gedHH? z%rEpZd0Z=b-7DnOU&!5C$gf_@^eSH^e_>zp!oQObK9W3jEBPl%Z@}y1ah0ska`AJo zGW?D=DQ_vGyz-yq^W@Ln{4ROfoh(mm_q{xO=!p^Jd7mKP*iN3mH+lCurr);@`BvD! zrawdXC;tKVfbqB?YMKr%B$s%uR4`HRQs3V;{E0CtrLId45qh!0>i(I_1^MZT}STPjq-J; zk$-nK%TrZPzFX_VQt`U)GW?V?DBoS}UvVaRl-BRX;=YraU*BZPAA-JE{ykqMf9Cha z{Ao6e&A_tU6f!_Fo@zB}8a{#nC-i2DtQ*(w=w-)pC;cnjrG|qzUdPz|5ov( zHA$&m-5hl1Jixh1GY<`Q*bJSYO94 z%3j(b`S!)kKi1Chx8VHO;t%W~-=+2%)JeVr{&AB}5%*#I8n5i4e2Mm>3ulsFJBIR& z;=dlG@`xYSeA0Fy!ykMR^Pekz7W0>-KWsMT<(HM`A-I4O@3wo(_41~x%`U^AG(RW8S9IcZ(9%f<#QQ+>bJ;4 z*+FN8nS@?R|e?!Ot| z@dZzjE0!|+bMF-Mj87`QzhH^ti@%5cx0k>8_8E#Ve(-n29R5i$ zAB_7M7Qf;yrhkyaJHGE+hIjmnlJM?(yNeb7sW(`j*>^L&S5`6puw~@8>M5Uc5BWFW zqP*u`@)NLEtUQj7`3HIZeU$(EESA^t^x+zh_fvj5;5zb&HI&!=k^DzL+hc+Fwi79jJxTeL`IK)GpErlR>M6>%oWS-Nyn*~A z-uJcnbGZ2Thbn#Xy3do#pJw>SUsZcQLmu`n^RE{_@fyP~700B{%bsQU@k<$g{fp!g z<5a$v$SXEbKH#t9wVjj?7vDREJV$(;=8py9n9f5MinAI&1N)fXx3FGXf7XZ}7(kvQ z_O+3#US{|!#z5KG3OP^@BAslm;H_Wn|oNkfm_I%HQwflpW2(@7raXOeg7tB zUMIhx_g&V%L!S2srnmMVWY_*}JIJp-!uZR?N2@$T-=+M)ne4CHvVop`nz@y{P<+~T zOn=aBlpj<_uKfi0POX>229O_5QC_!)So*L^y!qEG&kpe;Hx;uhe{4_2Kl)&{Z>RW{ z$>iAsDW3%Yr}bB__}wGP+r&?nlLziyT%P*<$>DQ2e=iUZ*7~$V{CBOVy$3M-8!K7f z<>lnZhOs=M1IeFBGQS$JtIulj@rN^f*+C5dUN^%JKbYL3{rLj%(V7pJi{EHq{u{*Q zw=w?U!Hhpi_YKW8=G9eDY}a|0eNqUnI{zjN$kFGp9<%zvHun;Fh`8^tr>|F`llK85KW>|=hl z;%Y412-LUtCv1o?A!x zSKns*1>(E+A}^gt`S7F3Tg1tinf}Il%5S=qJoI$(TVJ8ROc7V?L-{=MRq}5w79Svc zcCFZl^~B0M?hK_DA=S{v@?4L{d zkjYGM#(2iB6c6|lX=)h2qb}$jim6tC`+b@h#JsUgcMrU*r;opDMl# z=UG<1MdJ5x-@+@e__UiDKXex3zjpz7ocQdk7(OHZl>CA7#9vs*@XN%V2ay+?&Ga7C zeq@XIMAf%)3gunbF@2wSt=fCIc*4a@Z;W`5#?MsokItvON&MPua<6#G@5$T5PpLjt z=P>_es_zu>YoBHK1>zG9X8Bf&kN5+5@VShC>zNFn5&uB#v0NOm^N#`WkLDQu3_Je- z512;&M>BbtcrQD@fIK5k+W7@|Sp&n*Tuk03o^0nAkPmC5{Lgki0PYr7TuI&_KEci} zAon#hyer=nafO|)Kt5OO%C}DJ%I9lgd{@4C=aXG|dfJQAUoCd&ZxOrnV;v0d>enN7 z>1~}sxvSrfnPk_VRo&#%>^uqOuM@v?o)`7IZ_{MLzGdu|oKHk;`W znZ^A7J%ijPKFiL3P@Z1#gz4l>;=y;4%PwI2@7wtg!dHsFW#>QOdU1`N&wx9{uii@T z5&zK6XCMz<$n$B;orT0TqFLCozEcr9P#&C$wMz<`1=~k z!^J1xL9V=*@=vLJ3&jVjeCx%F?7R@^FTRA~ojz_9yZ+rScKthS4#T_ttrx%7t@OlB zUssBqzHSh^{@o#V{X6DTrtikfe6j2A4Pw{dRhKcmD_^(Rm1m*Y>HiwB)5lF>SH6Ll zGdB3epZ^V}S0g_A3UY(EPW``7e4P5f zSNuuo%UZD;Pdmh}e+PYy`Mdu0iSzv@cJ0+IcI~rN?D}(!*tJ*r6-?iapQ&OuerAhZ zfA@%8`BsZv`G$R+>AC(~D0bytA$H~6EOzA`G>`FJ|5b=x|J8}z_?;qltl@iO<#e-y&Z3ed^DU ztC;`o-zJX}|5@wX9I+d33&n1{^@{7Xo~;#MVdvMFFSd$Z{|vsG`ML2mM(p~dUhMRz zTkP~_x!CE)IN`d3^tDaw^tDHPy2i&6@uONFd&NiFeHirj zTJeDIlKaGN{vB{V%jeqDCwBTUM(p&zP3+odt~lR5Vz(YI5xe?q6}$Erd;{}$>5UP) z^g6{Zy+vY|Ua#2IXSLYr-#W3=-;H9|{(WLM-gbyxe&q{UK9`?Q?8;XwcH!Gf!uN>X ze6URH@?Rl#`m|c??7ekjHy$^OuezQ6Ip9W?&(+r_cKHn#yYi0{JALXDyZZEq-Tb#q z?B=f(VyEw$#I8TfzRB{q@>Pi4{5nSL`lG%ie1q8O!#uGo?;6M2uWS>$@(sR;`MdTF ziCz0<#BRQwD|Y?AMC|69Rbn@PY!bWmw7iG;x%G6I*v%(n#Liw?DR%bJX0fx6=HJZp z?rUNHt`dJm>*Z##YmY&vqtRnWxd$hAM3uu{GI(V;Jakk zf5XMD|H{8dxm#}*h~0SGB6j^ZYzf1={;Lx^ee4#y{+Ta!{j*x^+CO$X({t@TU+naC znb@`e3bAYd)neEF>%^}8H;P^R5BNUwbL}4zyY{aUyY_Dq=f{KCwf{1)TfbL{U3+d4 zyZNc=4(9LbKSu2IuR-kEqf_kaJ4fv5w^;1TyGrc(zfbJ;hviF|znlMti(UUVh+Y47 zi`{r!CU)(yLhRWeVHqE?Xy7a^rP2>ck7YZwb$Ug zn4i;+nAo-7WU%~qVHjAA;Y!kcj*>pF{w#|Ire$8^3GCPT%^(ZafXWhv~Wc$HZ8VpsnmKVkk(KSN^I z9#h0_KA0?{^KWbsk5>8vf5!4uX#Z3xu9m-~M!efHrZ+|Wa$Axw%TW__+9yv%6`uBHAp^4e4WlGhlp#`KlS2H z*T3TRYQM5anBEGtU%B{FwMT_`&;ph(Bp$2w86!^Xd~%9-h|+HpAFcG~iYws{Nb9cc}g5 ziEmeXFB9*k^w)}CQU7do;ng2o#P`cTG-x%;^XlEK|4?z6^k=d72(`x=@miICyZ8@U z?*>1{^ggZnbc!EW{_Dh}HU74UuTy=>e!=)ps{BL6w`sf&7f+S`j1#xG^JZ~e@Tq5M{hpH}@him%dn`8M&3>i>biWclt>{|^y=MgEat;yu+~_2Od`f42Blwf_=v z^-{L~YVp&OZxVl1?Ym8Uwc4Y64a;Nyzk@bjE5#S7y=%qYIxlVze@XL2xA;#wPn|E` zrukxt_#vJDEEoHA9=lpxsrZ}4bJhPl#4DsPRlj0+f2R5j7e69>7$?3#^_ePuOzp8) z9CZFK@gd7tf8Vc}{t)@ICyO7~{5@OziK`fXrFc;jd87DN_5a}CFugl9|AfRdHNVt} zU(@+|w|I%po92tt^O*i}@t4$|8^rtSynBbZTIXYxYnlHV^>@AaCu*NI@dMKLdE$fA z-b=*IYVQ@|hom3#9%uRwD*eUciv9q_< zmhcX-vq#H+$Nb!WqC)KKE1%fyUu(tAewr+H_UBx&v-cM|`8Dbv@ynVoR*Rj#b-nn{ zYL87~w_n&UcK(z>zi0WKy;CW6>Bq!wy`CyQ_ZHl@+M}#&)RT;VM>Bbqcp2swllMMF z`J{#9!#0rDDg2eQ9tMdnvz8Z2!NC zmfm@rDBoS>eN4Q+(*Mc}l;3v)!}p4BxQ|@*BIW0aZxM&p{`>!#@`2xB_zT7Rt2{4@ zPlf$vGxum3ycU%rgI-)rO}F+ZBTOFUWn`+_)z^~2;Nw=(?Wn9q&x5}z%7J^FRZ zC&8XJ`E_FZ{~b0?zd`vKs{eE1Lt&4Z{M0uo|F-n~2jbaEZ{aq|Gq4XW{O)g&zpMVb zSp2ql{M(d&S?&L@IH3Od$~%-_346}c`>{BQ^~^Z9o$?>5eZM2#59^P~5BLY=vu`6` zE&jgxXUab*f6eNHS2Md0+}k^UKgsk5+$a8>^$)oG5Xzs$dSg8GQ1V|)|H1ExPd0r7 z4>^qT_pJTFW5lzrC2##C@TfCdiPnO^7Sg;cg2a>l;0t~ zWCnSuc!=7^cQoUlW8)F&O%$KFgj{|M<+m;%KYScHYW<1u^`9sE?%?W_aUT`D z!pHFcl)gM99%S<&qKL`85c(S;{=11^6@o-yD zzz>U0SNoq4Wc(f1P~I-iUP%6?_?=erL*lPXzc-3gHoqYK-9k)n%QwjTh}TOW4-`*S zdS4V*D82KD?$EXX6{ZN_;fzXX97IAzMGe`-hqTh0^bs_(j-bCQpd}Y4b7o z3h~cij~M?ze7nZOR`F4|pK9{$;-A}m4?Zoz{2scNJXJiejeLuEqRlS|e~0)Y_1}8& zK=tHpRM%w_#*jAjrXAV>(bvZ4X6BbnjdZ!KO%kjyLh(dt3h94_$QVAQQ`+R zK3@_aHJ$a_@5>B-wA%X^@wL*|G2#*TG5lNNKirS^FVH?~M>75o@jVgalSYw$g!fO4 zx2DN|xA#-P)6XNHj_>nX`<~rQUVRGXQ(MSC#Qk>*-yojU#QZwN@8ErUlg|^!FC_Ph zAHmBHmj2dOruQc9vs?Jy%gB$}{DpTa8tU+3ApSeV`XAgkiF}^T*Wj{xa=_LT@DTCW zZ2khT5{GO)2d_AT;s1Omd8_zQ(?5%U7Ukcu@dlnOo^9h7+$Mg>=4VS!Jix{qc#Zfz zpgglc!$|P;7#I9X3v1PiyyT02s|Xq z^eb$=0#}L8SAW)svugi&;_YUCApBzS23LOZJ?h_0;+d-dcJbq8Zy|npj`^Rh_OBAh z%szsAT?6HN+WY_>*hs!e`S+ejo~rgMo$`Zhy+nFr#0RLo7Kp2RD6i>Y_;F6(#5u*E+C}+LrQa=HY4#D)-!8t>)?;w_ z42JiqKdQv5Z2gD4M*LINf5l9OKhEY;$k&M%DE~h39O>(J@hSHzzi!5ls=a;UUr0Vj ze7e0)g!G5ZV)!?t&pz=xwthg~Ab#4`18|S{FPfj0il>;p1bNv7OmDX0&k1`KpQ+t$O!t#u^^&Ii5 z#D1k;BW^Z(5Aw<4GPhoczpwdrp7=7e_Yi)ucrT^5Lj1bgf1UVjtsi~jM>W2x=CFJt zq;ECiy*1xVc5LvG{qj&k%mY z)&85r(0CyPI!^7M)~Nq^Re8{B*^F29`RUw0+*yFu->T|7YjG4N{2dsLpO z^T~H)o9lb4FCY`=*5tPuZ7{j*N|j@r9Ve3x54#Mfv(F8>D0 zbBMxMiO*Gk)QDeF{U?k6Abn~RzassgCq7^EUyt~AwqHVdd&N^U|E&@quJv%c_zKxe z<=3!$PaXn$dl2lyN64qz_mjYz#J@cn>4Ue6C!2i$E`OBaC!IifmH4ew$Ti}Zj)i_8 z{2cK@d?vy2nj=_}j%Uy|UFTPxFh|A6WUsyRTyS zA>tz;GkN!K8GiiF5q>|EPkfGj9{}7Verh!81MU@{yAOGdxb_u>-z5I+9+Ynv_uBpj z@yj1)eve*7t`fh#gj^%`M_8W8;sM7qy*6&GQi|@wXX#f48zhe8p z@ip!fN5ruwSf0Pa*SA03k)JFczdL!Vc#z}`VynM}Zxh4(^LUPUFPsIMe1X{hZ&!>L zi$81S1NVx@D*ZKLygu&9r>U;F}Wze@DIt--|G=5dZ8Da+Uavdr&^eW8(c! zq`XFapX6i2?c%ZG!@t1rCyUSc3d)1@>co>OD4!~x`Y&>WxDoTeh3^*MaW47~-`H6v zu0VL>4dSb}Lw~@V#P=Rad7pS$J-OodtbfAZr$_iIaowY2pLlbMTqm~A;v9fBU3(n< z!GAA6SB!mr@|hS5##N}7{b$=~<8>J4_8&fv=W*zB&` zRz-dcbCbzeD!*%${(!GgzE-?;BzXzSi{GGC{O_A6Zy{ff{0jMbk#X>%{B)^`L`tMs(-eWMg6vn@;B>b=v9u0jc3~!&!E9CZ) z{J&8mzq3UCo09siDB&AQ%C~Px{SPY%zh6mtFD%LbIoPj-c1zPL<>rEvZjK36Cx*?|UWj?b=Ua z{&oyo$lFTV2cN?!lD8mmVfZB_>CY_TYf2cOODT$fSxI^?l$3u+NqgC`Sz-QjOZe3i z{W_wASCo{eyCnXQ68Qlo`5#(RK75|4s5}!&(t8T&ZNT$1o@ekpi)SOA=kPp_=TCSx z;dueii+KKwXEUC^;CTtpU-9(ec^S_uc(&kq70=)CyoP5hp4aibf#*#;+wi=F=WRUi z;MtDnA9((W=U;es;Q2S6ck!5Rlv(}o?2hLXcn09v1J9m#_QEp|&)#_U!Lu)(L3sAV zvp*hdn{qq{;yDPXv?;Dw6-Lh8aro|C7QF% z={Edq>uhMxCNqiVuFhN-!|UO!_SR%3o$Tl=OC(dR?f8*6y=5lSOpMNUPRvegY;Bp; zne_ginbnj@G&U!vWfSSvt`-FS_~k^RBN;c**)$`OMC&yoAy*_eXJbphzj>`z{P%)C zCC(b}kF+J*r#H4tOEk51qI~UHzrWxw{>aS6&W40~EtBj_`u+7|CXO0Wo2VU8KOzuM zRCYyEmEJ#SmyXV4OD7;>f5}t^Vuj+S){d_B>`rO$j1lAf(S~p;lL`28QGd#pNeBJe zQI)7+CCiKg20J=4!H!uS{(uV6)S6B>q!0+=P)M#wF@Tj2}BPF>=D` z*v*{pk}&{WlsklM9BE!Ei2){{{TM*e>#qP-o&D zZONG}4Xv%y{r*XHBhDP37&BqQDU<#yg~(*lnaOlImJVf8x$yfJBAV`MZ_l=%3r?Rn zan$&FbfGT{9nGXd(NHR$^#w~K_aB?_nXT;^jMVh>{|f^%lgeaczF0ix3;BX^f9S)j z7HG@_#@gV^hQqlq`Xv;Lr89weU>8wC4ats%NPDs+)7tF!PxO9b#_Y@nTe34lr;PgQ zf2Dgu>1ZsI3PysNm@kSsrlcU4RN9kioBh0*&xe(wE!l~squn1=Hzg+3kEs8z=xaO^ zibv9^cp#mP#C$nlVOiWn?yW60U^6q5HkrEuB|5Ur$+iYe&j|{OBJfXGV>_~l)7;kH z+L=w`w;bkU{79UdZJ(CSjBaX8C7U|R5+g^lxkqPPvh8T`6K8d1@#l}fDviiw&cyOC zHiOuSM7F)XwY@CS+Lmp}G~$;uCj6}TTVmq)86!K+3ynN0l56Vj&W%b$5#9`ibReSN z63Hg$H4{lRKpmR0?PcE5+0ogRN~CPLMxqJtPgt6@qBg*qu)o8Q@bXXKce5~%ul@Q# zIs07Bn){ow$(F9Ror|hvf(A@aM{4bfmR(6Y8ZXEeNzef;$!7k|Y@TMs@>c9rYilRU zm;^U>wV*HY)B87dLAJdWfp%ICW9!DYz6Ya_C6Ywc+4HiHkQ-!!o4gV#Sz_VWWO z6|*``OSHB&?^5qOkXo{7XZLtRrSy+9ggV=2*&MkuEdSiqva7%HZ%8QFgdRe7|0g41 zQa0J1ZrGV&CX1B7FkDd1+)hD9wh5F-C76a^-2w!IOJ z|DQ#l^k0cQ$;%x|(A+u$weU=6RNEW8ZS9RSFzf8HlhYWxCUPp}_~WbSNS9Kws{;vo zlTksI?HC+M6BG=*dKgF(vrQ+K{=F7vZ~pJDWS&a74(puNhNM1zAu*0l&33jort7@~ zTv;Y&JGz=#==5~khuDhXn$M;s?j~%e!6e_F-GyvMvo~vXG&N>Ss@dg4sJE?n_!E*z z&&)vd{8S7Irkx%$K4dig~6uD z?1x10W{^yimJ$~O6A{Mv2iG(|pF(RsJao{ChI~F!;C#}VkCSvhFg80MTKYgyQ+;qv z{VMxMNY>|dzqdMmaGGAbo2p>qYVX9h8+)k_j^Wn-M5ev9Egxa$8aOW5f*szp?4-7A zx-r+7wjqcV6<|`fbM(X_99y>&F{nJ;xpxe8%xX#77AP|#*&fvhj*cYz17fa@*Fe;4h=_hxUow*^Rc1u$OnVA?DA7Lzz&>7Yq5rX;*R-$}BCoNt)5BP1(Q< zU!*SCj&k~ajYw%)wmlIEpr~o=B~vI+$mdJ>^U+zsE*xVKlC9IhEt*Npr z;H$K%R97QFbu~OJsKzYs&#+9qpIx?)SpkCxV$BjoH>ghZ;fJDWlDP$cAw2cqGOFCB=+U39OI5j1{RQ>Q0mmL^JlOr@x@%6;CU^LQ40u9RRq7o*%LW={Mb<~GyJ}sd2-TC$&L=oGM4lO zLcXX!8OmlOE_%r4H3N(~ubRn@evOe$bxkX%crYDL$HJL-D2`e~6{uHU$-T{x-|x+o zW1u(L_L!+pqO~P!{gTY3qq$Th7efJjE}}nH;@(0P{K2v2)iVDR$~7y}*n(cRMh`&^ zgSiO24@n1{atp?d30_+2pL ztE(~8OogSU>2x&e4@dktC~GR>l8#2yo|)=u8?V*XZLN*)AXZn`c|W}ZaB^PYtbDDKzXN+7(d=?LCdGM&MR-AHJxti%48Gf zeL=%SB9TBal1=7(fn+q`lFSciKib#kw89bX&9&9lGqR}E8QHWYt`byNdqqHvJp&Ry z)7X)v8>*{mQBERSjILn7%HtlCBpS&Elc97n5b%eh@Wil-iXA`kd@uL|%@`bw^5sR~ zNtm8(w@J`?CWhH2(V6U+j%1UOs4txhW&HklI+}6G?n>Qa?NYQDU2hY=is=4u5OYs9 z6b&XL*!_Q4sX}s#s9dfJVH9(v| zFdWHb;>lbv7>(!ScnT4Q{guv!I^ZCVU~R;5fnn2_>-7F&wTlKKa7$n|OZig~mt9_$ zqQQ@%N@nN1{6^-555URsJ z{-Euo8?hv}XLF6+h!uo~FNxVI9`I#S;eJKL5H8jB()BB^fz{PxynVfu2`T%1fwVsu zjKuw!U=%*gk{QOoi=?^Kq_a&;=c@LCs)JQY74fr5mBk!U2C4rkN-iXZg#Ynf?{%}tHaRrn{MfHRZr zIcT8$uyPd%&4rR|NqFj0T|LU9Nv2B$AsGh8E7jHG8aq3ivJ6&ORJ3v?>yL+%;Yb`V z+Gw(&O3oGxB6(E-jhHUNfM?-uB2B4(7oWepjTY3)9OsEAjXmmW>>Iu*Li3vHF3S>hOw0tU;Qgm;i*}@j# zx)%0!kp9rAb`)maT5mlUg0XM&9ws|~D3ro+OFHGtr;(o-V4Qx8VOKoG?$l`H1JfuT z&8D(}R5S)psmdDTT3otF!A%Gw5AMYo*i~X#ACXRHJK*CR+cE?GhRoPZ|GAcCVTmOa zO-0Z^;YcbO_C?X59QhF%eK4VunW(u_a7yP7j^Hn+W}8#l_Apk34l{{j*<>Id@n_TF zNGu6^iYdgnEI7@YiBeWqTT0c{S&VJyCdS<)?5L7OHruYkd)b;7)sC#JU$8)$jhsux zqA7nUp2=l`FpZRP90y%H=bOTH-s{xDW+}7@Y8{{PDIfEUP$8 zneOM=9Nh9aJI!{tVd`r%;|qpY2HhTrU_y)gT%J3JMgKWh(|uQGBi0N%Zot1#;)yNO zCS~Chrwbfg6|PCKSSXOn<}mEC5p2O#CmXstN^++-e!n-#@kqz?!j8bA& z2LGgU86S!!#qt_9Z`HzPPaDWga)&% z^opH?f3wMEPsa+Uz(R*X$;yDX%lZPoIPAh`EaWoDYlk03Xz99#v01ud^j6mDYHwv_ zr$f6xw%s=gYjxIhtw`BIp-kL|`w^LJFq94y6>e8o32+-C2(9P60ze{ZtA8Tb0<+5e zyBYeupf+I`3Yh>-(NnpQIVg5D@ZWDE=Z!1#=U_MI##ktngsGIy1+y7gUeR5q=h-jj z%7Qn>Qv{U52m3w~@rC@MR3MmqpLpKJEHctA*kml*dhbRL=Ow5Ll7ZVSiA za!+}>h-JCOmZo^h7svRB2ZMoJI3KYnx0YJ};tG%j=F%r^!GE(|;iSn#09e1T{n zhFy*y&e539wRD(MY~{q0Dp7ZDB84|-*Q<2NbQ*+C=CH^n{dh9r(vkQv7h8C2v8#zj zL&-4Q%PzHW)LU!|CiRnA+naGRhAQs5l-`ER8*V=Cy3*KuqWZAx!eQJN$-*%3Ctdn^ zjfqD3DNJcA7q}XBy_X-C>}p9jczK(PGaQa$7lIith8ei1b)fo18$%9-R45yZ#C`r) zCLPbfIQZBTHEkh(z$Q@`x88vSYK~11d>~;s?t__t-=%A#(yoNX%_8=W?fl`^uFkfu z&O{ELZ9D!$+(^!sj)XG4Y}}lHp`uz~wvF_6M}FA0u1*^~HB-SKelr^4B|K-l)!lr>XA$NBe7iAmrBKbX`Bf$6crQc z#`6jv2;*p@Yg(eS3-=PyNjU2a;D>n&kxVj|3gu$K0LG=^0{%>^nK#}U7&cqF$z!^JEo%nb zkw7MfStOv!7jGA$DLX2LSK2!W!FJzc>wt_O(^xVa%SN!_+;!xSVjyYY?L6yCW0i-k z8Oy~};YiXYYy*$A4usr_=q(WLdXn8F#K;RqQyE_>I3RGvrX!q!C7li8!cZW5Vm*F6HQ8ou ze+pKbGn}y=#^u8HStwT!OGg?C!%R4u^}9?z+Nmio8K>>mHr$!m4qM~m3?ZHggaWa`^t9dJ zVPSr^@zHm&CF>&&xq>*34Eb`A2-bk2fo+Fe{pU`v`SSBIhp0C-pVW@&So2W8mr6oU zVN2tFF3!0M=L!k_^rh0cJzAW$8P~r4X9lIKq;nzI30o_?B9rWPtW6Z&HCY|RR9EjTi(Q^EKUn@^ zx9SO_FdM+#&@8t9n0r;v0&hu-n}e|#b4|pv-3{0$!wH9KgktdDVatw%B^ZsmsQIuRssqPmi=9!Yba2q@ZJunN zY{_r;RKs904lhv#*K^?)@S$$rFct&%26VU%|76`5|DHg_2PZiYEwf z9~PdO#P$uo-X?g`al_L9wt5&xQ4w4gjbpwHy6E{fiuK<{v5Nf7- z_PJ!l7xDSi;e4(|=6=xcO(ppo0B$⪻nK<4%4QivmMvgk)SUUjs-(mXln+W!Cdyk zS|M=eh*L&QOq@|a;>2;I5I+Xn$QSenU>@NtsyM#qr;N0;b~ff_`NQMwXCt%@Xzv{Uc`T7djMA)IT1W7t#y z-m6sF=f?sAD>NB3$B6!x{*uToy4ZRo+5FCC#L>_mK3f`Mxn3+7#CBlz?SW zz)=>}CZtG5YgfCuksAVWoY}aX{j<>=t z4ril~`N#c%4s66S+CK+ElSpm@Jq{DBS5WzxBT2lo0EcHZiPO(qW|w8~JHG&r*8|aL z4i5IP9|!F4ttm=@kzf{1%x=Jx7&w!hk+btki>WOxKKNiLq~YCETK%mYcZ;Nbrn$=D zMw+?P&_SE_H*#e0;b@l55Nmno7(vs+h=rpWSBX#$?7Nijf5%A9VYvyx4vhx_8BCY1 zeF`i$JE^b6k#ut#R*IHRUOFmr*YhX|o!;1K*Bn~AFz^#t+3f7A)+3w~BUdoyEy?3@5HI-y*pz+|K`5BIZR8V0&)o zlOY^6YG$&r;rS7<9+WzRId1yf2)G-C1AX50s;;I|qIHs8ovjJn1jbc%YpfB#Q6d|b zjggQw4|K0MzBeurK0h!m6jUP~2`AH0UnJ;D;@XNU@CR56SfR{p@R}YME|5eHm)eqP ze=drh7*24C8t$VU%HiP32bTukH4CO=XgBqq=P>n7Rzv1#361MjY%-=|dREIlu zoM_eY4z;a7p)lTDgJ(GA3#QU8K~A7p;{9>&j)FEucH=JLT{y#KlVmg<@P%QnhEqk+ zLLc=!ER}>4I_txL2>LPat1j<jI}?(&2PfI$T*`$MR3#{2nPb*3qjGS zORZ?i{GfY>@ihEsaab`qG;7dBk7|DBCAxTic8}pAnWr6^D^uuHOuukUz%`tY?X}`Z zZVtZ>hNv%w>pSpIXT2WsX#~(`H_Cz?H!38Vd&bd3l z>kJ8R_QeSrr*tfy52my66r2-9k+?;o6_mdN zk87WH7vAsV)=BrC@TO-f4RaQL4_LD<=Z}6P1ZNCcZ019_D4kCG^6@d2 z(N13q&L6XI#9`&c9STg9saV0-DN=cV$b01;ej?Z~cqa_i_h)erC5FivI)WL#XjJ#R zgNfaMUBa}DP@*~Md7H4*#-dT&>IvcAXTZhB?;QOAKAjwjB=)N zqdNeHTeL7+?{a_tBR_9*Q+&O-*f5=#O=j|E)_K>)&XE?wwX$d&GYlL+sA_4JA7P~B zPrxSG5w2a()~p=CrobP!t_$O)x=Z?l%ojh+7jL(Z<_Kbs4GTXPOyTAGd;5jOLm;VXdcQFo>|Fn>Gs<%NskoaOmcpge7n{vfb{^;ay168t6*^ zFN?=hcp)L0cBvJ+g$lgc?{h=i+`V1{!qCFaOq2>QCxny#^W@_32yTO=(zy5^#3qZa z6W(b-;x!A5^Y(~osujR6VSD0CqBWPpwJ224R%-YuvBsjoa3!i>m9>HKF%KRyQM+K2 z4B^h3&*$%#T67mz?%!`CU=EG@jm#@0`^$L3mR#NxT*mj|H+}H|UBs-T&#y8*WnJ{5EUOdNU|X z$0CluKDsrScW3Fq(R+0Ux?l3@Olh#vi^GK{%mu9who>MDPX)uU17b;6(&CBHvmd-K zq4?tyaBPt=8=}KT-6Ts18$29VWD|Dw=uX_+t?pnN6F&|s;8VcWS+#@TPiOW=F7mF9 zUw2-K#;b|)AXt$qyTYkTIBD?{pTu$9?Oh#}*WrWh$-u6ClUQ$O;JvQuYP&4#<%7Kb z?^OWXLhQ3ufe$<h3wX|Ehunk-~|LY#{B*ioQ_;j zf;>9kh1WiR%!4!v-fF{%QvfILF}Nq9k$f_=aQUx4j87?G1BJ%c{*w$jUH)7(J1{|qjTsDMr9ySqUepIIlKJgTc;^GcA^2pf5juka*mtEkugH9X- z<-B(Zo0?8)u`%;LN8CBZa6eD-*i%JI#0s5@rEUI*C-Gt-dS_y&`*S+&#O+wv6Yd7V zM~*gDn=ZBpk7aN}2s(uQe-Jv5FLmDC5bXbsSiciPv}-mT#D*&aGd3AW$6Vd>sl@wx zdsXPdTcf4!3fxP==M~_qb#HT>>Am;JuP$`tr)2nUL)kbkk*700Ty%=M+-(5$|Ad5(ipdGBed?5*)F1;13r^xpH5Mo|d+Kw%)`W@P;_9FW5WxE+H#Tsbd54 z7%~oqaBV0aO#5-044(>A#{WSx;Cuof#LFhbNvy^=jOU;&u7Ud?VE%rZKki;7vabiU zws<}$f5f{hgsVvf-^YmOVsI_zf>C@HE*f*m2b)@_;q76(IP87h2cO_<#GmZ-QM}lU zNj;WM;fh-(4OtwwWN^>ck85m!e6;>k$^YlytMon(!et7h zF8`Gb9!>1(RzdA9zLL?%xBK;ORWOzG;c8kW<-;{-Tz6m(?(}(M|I>o&eR%)fJ1xK~ z#MtfNYZVQNc6?_HZH}Adu~-lvNQAMAPh}{9UFja3YN;S@I-2@rlkm;B@C8mE@7OR9 z7=^|=Z14G@re;7*V@J?wGx8QKSI_459zA%i#EVP2YwdeQF z?ReUb!Y6S6p2qh!f>41FUW!xgc?w=^5O;K?FylAk)zIqd@vW2cR5&5Ne~g0gw$-q2 zs+zvy-XVNn_8pZ#6mO|9`{Ki?fEEkSCxke`R45ot!|55r#c-dCleb;4$$Y;(cClB< zo6}2en1b6{#pi6!rqLznqD>~I)vOPv7di9*UjN1g4c3E;jpq}0AHLuTuD6xP*K^_1 z^9ErIH+emesPo)gHDUJRd>efWdmveqq<118?Eh5|XX$C1hl2VAw2J=iHFuK;4sbr~>8x5vhLZ@_Pdy{T9*n8iC)s=nTAF8y+Z=ZwbPkX?P$r%;!; zxPHJjSZq`Akr}*h9>lxmxQ}0?Jo!nNHq0*P25-qO+$g*Hy-x#&)imS#SF(P2!glo$tRDzQ=FYw)QY;xKG#+J zt(UGC`pCN`Z$`R5mjBd%zJ1_bO~n2;7YW*HrZ_{xol{MEc}4vgU!D43pAv>K?h8hu zn3$uuV~sPP!e+GPtKV^k_WW_oKK5yBeBuW8LV|E<;0-T)OawlS;xz0Y@-8m$_H&u* zbvMs@_rKvYx2oc^F`=B@9!o~7aYFB5QBf3I_XA+cRjP{yBT&> z2Oqh@?1QuDeE8ypKW2B`JFtJnl9;yuI^q*C?|JWj#BgSoz@-q&KZ^I^!ohelf^Ssd zWUxq2Wom^>`ljFRr?6#8KSh?p8%Xx$3o|pYSuLoaoi`WnlOrwOXXC?Je3h;}>g}0Z z?PUwxi%Q`-3$|2gn8TUEG8S&>3Ve{UTnLAksZbm<81C#8vZP>Oq zznrG+vn}25T#b9`n$qSINgBtn1MtnNlAqPo)C%A1p=I)g0F`753UTGJ9;$Kpv*Vw zzAwXjeDs%anh4#4uhqxS$o%?S?Pv`@$yw9*ftZyY;8IpOV7jsw|JI(xa}C(3HFZWM zqP1g7HqiuRdc*_DQkqjz5amgwesr{2pHTXRcs$FhJniA8!I?RJUY?>V%knkAz z&-8eB-q6w!2j3CkRia-%P#dT`1edWwqasX}=eNds%m?V{y(&0iK#hSt?cC#t1yIBO zr$JtjF~S0r03V~hr}N{tKRQCKP3c1FJ(8-%XU#dF+m6J*Y?rmmnywFB#0!o|($trY z&7z<_on3^U&?w7*<6mKgdE=T>q8?N5Fau0GL2&=j6Ik}eD0y}19+qO-MDhn zj8qmGnT{$OyObl*&o zeD9`2r7J%XlUE!>`MDiNvs#C@oPm9(8t9+tu~9tRG%M+`#8#zYkX5QRQ33)vaYw(_ zaQ=ZZldK1*gfn1$P+TT;>7T()GSvXL*7Y04JL$)v}C4>Qf6~Q@oN2!3-_ixdAvOiL!@8~P4^Af8+X6!fngln{= zY2(gpOL2cJSh0X3=a%8m`4JCkq1UD47;}-*CoFihPHH7b*=PHnS8| z-A)Lv-LilDu?>QN-IG>i0*Y)uki0+y0jLB%dx&K+g%ttlDe*!AJ!yCr8REe$67Uz1 z?`nrsIz}L?u=hDwKDUN?D%zJ)MHN5Em&4WVB}e^DN<_Y@y6ekyq?NT3;=$TamONk$ zwxX9(eCB?$QK>BB7pA+&t&;lf&p)5OJUeBv0UV}t8dT{3ly^Fq8rnFxwiG?Q2Ou*! z<|vUtIN)wy*|$UXfD(SQM>zS?DpzsccVpq8{D)Mdtq2Xa*CJNmAq-G5j4 zFZcLmE;(g*k?H&+Gz6XR_KPe5IVK(ezV}|1@K7x(CGs^ z&3Eu^k&y#SO~oQyy%MAs(p=8%KnIYjNohIbr01x&H#u@US}C+u?Pk34OxV=0Ng|N1 zgoYsLOFza$Q>h%kP!=j*%72C9RZBkHpM?BSvyH}`1a|&fUrKH~2a54T=E)n!iSq9H z{1Up>Z~rOuk+Na6s_&YtGERk_bOoSgispXI5O%=`vk1Fp$#vZ}uN)oX5n~;dCc;lS zAj0SK8`8fHdGb{s^Q@>dau;pg@>B>cn1ZL^z>AcM>Bv9641T_CNqjr%F??l4CJ`+)apCj#!GSSI+$^2amyQ$%P@bq;q=ia^K#@I z8G*57oFi0UnwGrY{VThQx)?ThbSUs31n`vex>PVdq!IjsSmH+bsty^}m2>V4`rZ>bNc9tSNx|X{K4sf&y0r?e!3oEHR6`4B$+TXL)hZf-~k^HsJ3F$p~%DM4Nf5YwLek^VXbGkgqKrclQinlOld zKVOIqEmU!Oufk0R>_K;WowTfvculOJ5-Hz0${C(qGQvbS83g+yJz(zI$fTsrf6%!T zdIFUNpb)t)E+&HwL7NY#zMa%g*fmx6{PZIcTcFKlI+%ZVBCJn4aY*fM!+ zehtfm1B-}hj;pdC4NfhIm5x+@oR>5oxw zBk#CE471NT?m)M<9L4~nik5C;djM}V^x6#H^BY>3%%kxn_S^*kIFAp5Gem1eOGk8p zlN>*Olo1O$z#s)P1a>4nrrnJT@?M670*FczX``wg`{K}$q*Y`Tc%gbznB+<@>-1(1 z6=`;=hLbCg7T^tZ4|Be}5CT2uqEP3mV2)1Q3r+nBJt;i7lz9G|?Ke0Lr3^SoGduZf zHZY!&zx)X`E*kvsWy^x@1FCjmpb!%+o$e`_hDtaZN?!!@LR^*bL3#5U5kiFlWC-$km-%PR()}`bAGdeS{ zhLGl?UFXL5{ymMFUHFR#9tpn_uB(i)c34G|f1wEag#wehS)M=_nU{Xj1qcwzUf zdYfaL&vm@y!=Sz{~s)k54EJ2{9m%4z+za&;3A;oVScPu1A=HR5)MX z58^UI0$%sD!9TBfGWWK-ptS-UFTrBpp)@}DcYVq(tbw&6s?bAuxotW>)a-3(v35L= z9zMf_!)G%Rie-@DkvH&HitSe@n)xqMD!1Ao@wub2lw~ok4N@>e;0m0T(6I7Ww&SLD z(*yOq$IYjad<~MtN*>JvQTSuhL8Y`q(~`lnQJnCAo@l!rltY&{k`brec4G(d$_H2j z>LZU42qIDx<_ofo{kY%7JG&ZQirR)yp6S5PmrPPB+#M(Q1Qae5l|VV)s*z+qzrX zOADfGh2KUedRl{7t*|)!D3!w?z~=sHig=hKDk67$9eH~~=Fg{f{IASf-|RRk2EcJp zw?G_VU%)JxMv|o_g(B1f5pPT3VkDE+QnYqgOt;2U8-RBmNbG_<)4)o>u)7tnGk`3a zU}`H{o{;L%hqSr7-8B%y-rNYliSA*bQR4u$A3S>g>UA%-{2*$SCB|!5<2=ABPX4ya#lkB`9w^a)>U#yxEL?=ahGbSX&2;3VRPmUw@OZAK;N* zgp>1FM+;nku030teLEzSG>jCs6*+zm8VE6bje zo#Jm8BHYbqg`YiB`5=DyP+B+=OMD1?1+KfgB8!2%P=+__z9(fd^*6%(U5ZyZOqdCg z!Xl$dh^0cKzbP(3TAa>DeC&{&exS{-NKdD5-Sspvi8meD3@t&gWDmZ7_WI$k7R_*E z_(&~4!7gl2oEpjWC04Rfk77P<8^U`rV+D+tL+d}%vO`1XOqFd>A!lqmgV6kBz^kW56v8K4Pa&$fsz95kv zp3XvjQSV^reRMXo)s1R^OpC=5J3Y z%J(+O!MdwsrW$cnS$H(W;%*Ryeo(JcKySF^-HbOR*bP@Qpbgg$DCA`;TCZ**&9!?- zBMk|8%JYD4NbZkxnVF%Ox`yd}>RsT#RH0UqX#Cy{lC~C6KjcggoGsDz3Ul`;0u*s6 z9?c&T-PH^ZsEB2UGgb5wiR`ffl}~)yp&lld)h=TS~$y~4&u-K~SXw`=Nfwd+=&#}$UuK6Bbb z+BLlN5{_W3Q7jNjb;@(>4ib3o5R!>e9Kx~y7`~MAVdfs&*?9d~b{}s-hJyqn`ieN1 zaLM|4P)qgT?lH2Gv{B+sjYOk`A4Cq>wQQ@h=~3bQ@A3LlQ{SA}O4SA&Z-7dSKzN6c zO*whd6@Ow<)uI6o(VA^dHIdw!zxL_aPG=u9a+9grvvK1^P=BZjD*32``a3-#mx_{b z2fkzgheIokA7}|dZ|VFr2A4pe&}!)JxD&84U}OeD$`R=&KTg!nTcPK3J$^~E=lbh8 zy)Bm)4o1;KKSI9^Z7L09#{S}Ph*3RT#2u@rm}cjqo-eO@VQM2WSLb){{hD>cqM`te zD^Lee3cQwEMNvC7sIA)+Lc5z^`HdJuZEMa-<^J@~EwvdYS*Pk=04~T)81y(e8=TF- z-|~m#O)yvlh~S2h(&&4CtHx-p)Cm({*zp?70Rnopu?j_#1wR4dR%EPucpG7o5M3iT zr_-|UjQyRwa9=X(klcX!@t5erii|=#Ap#)mG@8hg7Svs`1%ygJd+9Ge8n!nQwkyQs zMms-g^bzw$Tem2!7UK35ejt4cgP44WEb|&thVnWTDku$+L?wCXCTyKH;e$9pr10tc zBELuv6SNR>8^7!*zv|8fgzZfIMxC-(U{n7%6@;s^?l6lB@Q@4)I!s~MnKzMH^)LiI zu&`jn=_Lnml7@?1FZKLz+qW?pm5Og+5<-Xq$EWa1 z#Zhn(^F{@Otxvxu(5~oJ=X5NFwK<;*n;+McvVvd=a#>jC z$q?d~a+Hj>kB>$n>&_BsM2EH;I-GcVV+-l((xAsLZyjH)e4ApV_|t7kuL>`kyLdV) zf$qiRqYLnuUzuI;9moR9AP8TlW!He9BRoRO%7pMc1Q zsRjm0m!DftkI$xb5I9$oejx7=uO!dhE(Uf0mAY4>nWigVAZX||UP>!+l~bDR zVU<{pzanNjH;YmRSqUY@>te)4kj%ky^(2zeBJS^$3%78lt8n`3pwL)%yTlkbalWL5V% zI7H=aBjmpjsA8$ak0~JtecKB!(ap&JZCb9=%8mzU|3ftlyy?a2&%dYN`;z=J0gdp( z_91bU*^Lc>0ct%ERyzY?YdgeaPuWU`BN5rBWf7D*bY;;cZ|(A{L_ucSt{xe>W?Ry?#6 zy(s$u%|dT8T%%vmmvpsd9fYYM^1R?0JuF>&uaN2c!|KXsirUnp$}iS_1ggQF0)u z>xZi?Wh3c~Ew4*2=XD3e#Gj|Gfk!ik@dX7?dY94MvA5sHec&bM@I@E*Sq1iAW2Z%T zueax>jdHSVk$vAjhA;H+;}5Q?@U8qc%FI$u0?y#>>VEnAr^_wXmc5VEQ9&lH3$h8D zw|tMohRId;nNY70{oa1#&j1=UP+r_oL>(ZW_Y?FKBn0w$eFw@NCkFhKAIcY;{N4#i z#(&-+a;r*VF`UCDR%0{+`3ay-=3#wjt;cah`#ET|Y}*pemHtWMK-Hg#>MKZZDj!hX z2Bp4S-s?!(3SEnVbPfgO9;@N+RXk+^3n12DofwP&foN{FSnDO~Y&Az=o|Lt}(l7rR zKknrp{iwB!cu5tU%9K_>W$&b8#qL&_S+&ogBRzZO69gnVT&iF>L#vrfOb^<5u*yn> zE8OC^odYR={$RO-+J=B2JeJMp-oN2@k^vye@j$)3v@oP5cnRNW4g4REkl?-Qkp=7B z41DK`H~4t~+9{D{T2aYxH?cW*NT$3J^gMMWN^|%n{(7@uOuC@EN-HY`2wAj;e!U_m zJQw?OWll$|)xl#(e@Ou)4Jk=~gQHnZ^jV(2-n@JM=GDpO0;Y*AFzk5sA0co8-Z2%VU?|?BCLJI&qj;QtsA<%2 zxMEbCVBAeUPW&OpL+v}?RtUh*4m!fVV2?Ii$M;n$~N$|MD)Ht!5AD;j;YJoVTk z($6+wk7_14Ztl28NL1W}3ZN36VSl~TRnRk1UQqA@ zw;zAH+dc5K208+Z51NF0a@w-pQ<>FVsTDR_IW!GoAOhsvNTZ@8B!QDnnpuxns_Iya zW7YHnUGew|IJZ!`Qb zX^z#O<>zZs`a;Gi-*yxRLCUQ=S~|jwA9+@Q6VQHGvUDR-mGOKkZmK*alPxlo&z}8y zaRW`s3BMP685>j4UV`$V&TmOb}>^lH9pUeJ%Y^#W)tCY0AumV^Yjb`LW96tgg!hI;7C zN#oX>)6`k*;it>cr(x4pYW@Meg0HaAOfW_Xf>sXsmZjYJF<_XSn#}`MYwO(tXBb61 zA+xf_B>W`Qf-jhd4vKHlU%YdX60VB;lf>NmZ^C1v|0u?dMmy5f$ZA8_9-izwh*|5$ zzx3NEOeji~b!~@I$#c;5Llg9Y^i-U)> zlJ9i+D=88v$OVXtNXWd0TMkgf-<2N_Z{Nkb)Qx*Y_>fAYLIAl#yf8`ct;ARJ{MduB z)ABYnWaUXi^DQwk%x$x~t-Q^oWNgK3n#!VL_RR?mGQQ{q)|k{ktF&wBmYR%T2~y`u zbUgG|aY2;HgA;wkuPS=dXgQEyouS>rL~6@ zvn@{{MFW~oAaurifnTHaH+*~ozJ_y$u5=jh1T7v0lK6@x4SL^Lrqw_8oj4^B6V%!3 zj=&FE7-klAgQL zUt>lb*;ATktxxQxR7Dt7kKqJysqX#p7yaZ+JmxPHnt4s9kStOn1a~SQek`wP^UvSp zijoAXGP##Di_%38uX9t-Ztm7s9R6tAX;lBQOlDjfj*k{g792@%Q6y?B&P2gP;N`g+ zPCyaIG^Qb@g0hVaQ&R$0@s|gqet7rC{>l`Kd7!(}?fwFS3?x19COCjAf5Yhsz7PJr z*V8w&3)NqWM3_AC9&*L%PLR3X!q&-D z`#VJlryA}RM7(bDQfWmQuizvC zYgPgncXyc?1P9T=m5@PgPs~j z-wVUkAn#1>%3be0&MDtDD8!jARQclU?fX+^rWWn?0hI&#L3b4AY-VgF9?3qoPBIRf zCMa_0BrV}78G+4$nT^MYX|wey5ctr9JeR>mgU|*Zxu`l6{<5bVm;)Yz$YsFpfOkz1 zDf71ph8S0Ak$+S_V%my$KPc#k^z+(QoM-_OvpB~=Jj)M6FxRjWxozivm6^_Jpk!1@ z=*uJ8if3BmW4K}#xzweGfj|1@?6HUbUZ*3#3n1I30(c**Z1m9RB7;gAK{%s&1);Dl zF^z?^=&8(oI4&zG371gZXTApNDs)uVSip4X{a7=@IrF5Nb_?R3PWl`gp4MNZI|Tv7 z8X(j4=2GIYfEy}au2hE|tn)eFL}UtTfp4ae z=-}@8VfG7-499_wwBq<5`X-0|{z>0z2C*gn2P_QD6_P*DB)WMx-~u-%%#UgSl;jw{ z2osq0hG<3r`-uQhgE%KZD6-c!qm@;??G@bKFsPZpi3M_IPQt^^QgBkhyWl$n_lBaV zyGK|Of$p%;L-=&Om0X!>8=+uD;tzj`ieSs~jKBAT!SSH(TABplA)v_{*(LXiac(*{ z?;+Hwd`URCLiy(#!E0^F*dbw%^D69bKa?y~LA9y>)Eq8(nU5iJM$ti!Azcd*ktF?9 z*PHG6E!ZOMih0kUWuLl&7`$Kzh(gqnpgms2x4(zJLZkCERnXiAt>C8xaHV&z-u(S^ zbMpM;A1`@1wk*wAc?;eu&rzbVc?zG&VxWoFu%#-7Tm(!cBZ3jTF1|v0$d|@0j+LjI z$pabuzQKlhEI2)%7jH=>$xc(8b>P7yh6>QBOjF2 zFu6>YvLpbMgS)i(31-5Us(CdEq>A^pH0|xTF6lOEd;##)OOb*}(oH1j zu)X~EJycxs!6*OBnm+F+%EO)cYGJ=S%yComA4&c&(GT|FcGT8v{EBCyyN$0 z8X+6{7BxlL;Ara%TawG9`F2oGo2BxAilqY2ES*gXc4?q@~^AaXmBRn!_i(n7X^3DtSo&4zICYNR! zoG~h5{1?cG3^2E`1`gFmL8^Yy(**Opc2$%zpMHqBW!4>H%aNK*wUs77=%YmVW^v-J zKqZ!|>wl(Ai#L!rVKZe$x(CPQstb*17F zYCa2bEnzd`+1Pn1Lw1a5~Z3 z6TkN)+jj&}LHQLan~Ex+v>H!_&?E_Iy?Oj?bF|#o_gE{dE@865{+F%$J)6oV14nj& z6+$cO2n#hzHx&En6^hp2E!)ovWK%1GUXplk!?_^V8E^(%?I7B>LP-=)u*}W_5YAPtF5H5?qO{Wc#Mz3BO@&lg*ONC(XJOJQnrz-^ms=75)g4IJS$>N8+N zg+glo?p}892KqpVWlrT6Y5mDsB*CwdZ@9SKkk=IUH}Gj6oO7hvAC9dA7AX*FBxU`W z_Y~adwfO`Pfsq^#M_d6Dp5RHPb@cOit&GyrutpAgKp2!@&=2o@#HXqg%^ECVT%+Vf zdu2z@cj|9a8n)SOc!CrjcV9!^Gk@4FP`j!XN&T=>yvi1?9y)ggPEmuGauc5>b_&fu zXkmIN3APor9sgm?cn}F!g0TXGML1jLV>+6mCkn=D=ec)b_@2H z1bqs&YrdLAerLX%A8IVzcX)^|OJ4YQ8uO6uy6&z3wpskmuRRiPJ%0h!3VdFt_D5NJ z>r_sZw8`pF%!AasoW4zDRhv5jFvGOeK#6!IMU?OxW`s4>k=cZD7i<>PqVb8unhANf zy{t$`v^9{OP(*BZLK@jjE{$3tCw#p)6nnnj;rxq`-3p@w4}wG|T^qvC0h=--&K@L! z;8OMw9i!Vj!c6(566=g;b|JcuBaj|ni0Ol`aQ;c>3)(kV(q~8m^j^_X+o8=g33odD zin|1wn4X%K-Q?i`673rWRCH8_sCJG1dOU!ek06H%L8uDi+R;4n$>eV~CdT8LrnhAY zk2NeY1QQt!GJM3x|Vp}=Y!5wfZKgQ$nK#16XS8s%`fFprj^Z{vxvHY5#I<-Rdbi}XZkh&E zFgrU7id7Vbx9-`(|sY!PQQe^G0_pSM#7Kp>a-Pv&9rNe*9_P z*U7p-$_pCg8u#(y7H0AEm7;rv2Ju^3-h8~g17CU5LGW#CScE4W3MmSMw4H^KWeV5Z z@ONOJAPe4&P^nho=cgGg>*ZXe+Z^#Zb%d<0futdw+(5VI-zFUVG#uS)#Vw&DSX#(@ zzg~>P8PEgGdU_X$Pfmn>t;N3i8F0|4`KSr^we*heMM{D6SwngXKb?Q<#cE3nPGIc0 zr%P6eRP&r|cmWLyex4HD&`^9ADll6xGM4jxYv-Bs&9sl=?!v_YTue!Y?))C0D%}Gx zxa=>!r^*wPl+&q#W*(zKn2dB$4>>vAnY5JIXz8*O!Kql3=OV(vg)sF&Hv^m>YJ?*5 z7lzN+EHhn)Vzl7AxwtDTc!dI+4^co!f+Z2b9S92iO>O;gvR;0cIW#DUws*aJFSA{t zM+#ppjR))_e^GeSm1^#M#n^hY=Rdm^eo-yEF*b6q5|qJ>UzPxX-Sqjb*?0y|fY*RZ zkt#p2U-z*LV`NoZWN4T%4606kj3?M3XhSHIk0|4nkj49fRy7l4lcJf3<^Hk|O@y=( z%6Vih=!+n`?%&B`kona3J#%*b?Znd6N1cQPiUvNt7E}>T%V5^S&BRry_#fU({^qUl zCP<*rq9ewcN`pdpDD#NaahD!!!%|;DMd@;k0!o;AMD`W!huy#8N(50FNRfJ4OKAt8 z!M*bLG4JoeC1q&7ko2k`vOMe^tWeU7-P4vX(08pt>^f}6*Pkby? zRhznij0D~o$3})-Um01xm^c?nN;a^iiWB3C9V=&9Rxh2XIpgP{bNjz z)~Gm?#qoFr1KsaxU9^`dCpW@~2p>_RX-TzEtSSA7KZpboK0pvnEkL0LHgSKmU}R~% zTb$}4)^N_hA@w&Fn3nfp0 zD@jB1lBm$~2j6ZqH{y1_i4*mN;#j;8pFQ*E!9OP81y=p~JtM2H3U zP;WsDdK4DK+S-v-Giq}OOXu;KDS~$J!#!O!{Xjpqzp(*#I5&ly3GP7<*Cfz|uqgbE zXI56KqDB4dNM#tU1*BAwumElWh{)a2JC{=|=pp&w-4--MfvtwlVMh{wwytpX7wgJi zV6A@kKak>5>p_1gepF7;+CBfF{W-6Wr@3=KthnE*R+hY<_zu*)q=6^R*>hoJ-Eosp zTuTxy`ZVC-lu<_hThP5NDK#~yD5!I+RV2^=%(^FB@s)ko?utw+z1V^u*`d=3a@LPw zyPs%pwUY$JZD2o5iKq|46#klrXQZh5!6K8O#=0wMp^H|RYgmVa_jZ4J^TOZseF16? zV?`4f)jqmj+;u;U@@7Ycl8h1ATGC#eCbobNE`Z4is6nTMC>(B{u|6L7!oW>Qs5QqdqA^u-zzs6R`liXA@F`99gio;NIfxNy25}kLdvN3yR0pVXk&JUs zq-VG1>hkV>ee(9r*~|a)T_{xO?gUH#mrDt}JoEPrRxzpwpoToC`U#%kr!=!(>?=At z&s7!?;0p5rnr;9HoBHQhKw2T6p}D5VgS^oV@;w(H#2@mW`Fr`;g;ST8#8gd24~me4 zc0xanewj%C7x(U^g>!oFI@ZolZS#Bf>;kn2QC1m7X+M3)&-#Wsu@fh*I2mv%lPc8Y z7XGqVl6erV7>QdRaotPUGj#_9iU5GpO)(OF1C8GjE;$Q0OX<5E6!B4dLVl5(exR|< zOX%PR!*p#z6)WMP4G>s=mMoKP(PV!$bzB2ibH}lYJf)+{|0EQ6h`{(f0to)+i z@~}C*29T|Tk)f;?+av9hOGBj`4sC!)V{JZ zWm<{Mkixee)P*(LzyJ?fGs_w3Z^~(Q^AL8Ev0H>k&WLMbN85xfr-MDc^rwdVktqi2 zFK<;4!!9441DwxQK|zsrRVzh9x0Ch24#650u5P>O3(V<~4XIu_dU1DHRTbQW{%J8K|{V8R^c5C*0WIAaEj z4|)+}e2YR7#c8VL7|Rwn@;8`R2?$BEj}X{Hd6RDk1bm}4757cn&h_VtLNqF|qeF)5 zUAQ8$lu-X%gGPd8e`6h00yh+@?z_?b1*6c6fXQtUG^A+>39RV80s&jW$GLSH0MGM& z6#EC>iW(tPc1kk-;_D9*Sw=GGY=dkWrObX~Ue_P8hT)+vBV*Q;TlMf=ziMUwdh zE(qlegdjbHxrHCg%WB()w4DbYRCKC2_CQhuQlnRg4`jQHob>rE7s;PG&O(&r-S6aW z3eltCF}jQv4C)L0Sd?@-`bm9!u|1(F4So~42wAR)A7~oK)}KY0u7~?UU_fd1i~aE@ z`BR>eao`wd)ri=cWSe}+a@a_!PQY$0j8j6Lj>imS*54$UvlVNmKEwSDHG9{B-^q_p zm@3^W-~nj35Z?O?tgigeP1nltU@o%sh#H(IYkmW@h?0{xKZdsR)N`46n-KX*&00D< zGC|Y;gcx!Je%*ayO4Mm$bk^s4ApK{CVFtddGC|wyNb*u%rPDOpR`GTYRD$#>ZY3c= z^`HW?{39Q&hUF6lFb*$V0?7WONAp&&d?dtRVghMpFT@NI*|w-#PR7Tg6ie)9SWfeR znQzv#6(}=0T|n^yi*TdOV^T6Bu4g*UPEX#xd-=jRIT-++r(F%DCbb!I_S^a6tfVq0 zJx8k)M9y+iJ95XAxL0A3rWuHgIspX(0>)nDm}gXfLKFdxEv9%tN^Zv#Lxlsn z-CsO$#?rY<$j_sQ3$_2)Ae!Q@Sy+7!;dOmhn$_w)%Yj*OP|~VfKpzhL*AK83=Yx$F zh)LKF3AECHRUH*@WJXahLrT>T@`O|+2nT{G64K)i+-n#}mxps@j~MaSmtR*H zkz+r$P6}iqGisCA^5yse;0w={Y_8MW>%0E8GPZuP2nf$~EmY4rI1&`4;!P&27o#~} z3C^vYvWF9)#~H^%e4uQLXL(b56u_8K5n=V7PyX`dlu-^yz7SN7RDUNgoe(IiNYJ1K!Ig}N6#wGU_Ysk zk4_3?B_IgSX?~~t5(W|@wCdyvR-!6F1`0$AwBX)^?ggn9f;DU((zi6&jV&d|9m=)- zfx}t2!UP+C62ASO^YE}wO;KfS}ZX-heT!PNS$V%p#3v}RVtDCoZ$!!YR>k#3^j0?-E-TbsW!W>C=;^8EV4ouGDa z4lE1rb@HGtZ2`k|O7|Fa%xLh1k*6NqDsWm@IfU|L3k^Dt=O0O@|G_X8DmHXyR2f($ zD7O6|aXLf`8|pWAXy`GOx54c?>uQnXEZz%9IagAWs+u z3?>`q@`cHk1Aste(xfw&WA$~~_{+?%`9Ok3gP>$4D$ZQe7wT5nvu7Z* z`5{!)?5E6|nkP7Ddt?2`8705UQ`*lt3J3dfNn^-A#XUnOi`TJ7W1+sfkP1@t^UI&r z&I3)EP1r-`Z#$nMR{s}(!FrDRCMT`ZY2E2DQ;ew0UAg^Rn6>32`WC9|kr`oz-XQp9 zT~zP&Z}17j%X$3OZ{Fr|C};B8!k za5h2Rra}sILe*YLUSGV$hzx1gTenFPgGzOcuU!UffJ^_DYnb}33Kn0eo1d@{C0;7NHC+;a( z8Z=JP*lHR=ahMtWP!2cs!)988XYo-|nBr;M0sYsCD2uZ%vkF~m4LURf5&&)KTI0%H zN4rnogSBESSN_VUQ)A2%w|k@${3OWv0xhfQI|z3QX}T4vw=v;-*+D3butl?rmw<8|9#n9Zh0S_n|!6YNnQ_T&99WGPgh5;lXmm%uLE&b4G`#UVA7(Er}|N=kz} z0RK(u2l!z^ybvO^W5%O$t{^fs`DPR7MFc-f>+~-@t_8pr8nGXNDl?diG&VvD&+h7W zn*n+c7Db9sXf@VZ(IT|l`L_;>3__X%`L>V`4^?d~PWiO=J7LSzxaUAIoAW=TQ@f$F z6}vaNu^9|Us_0Y`tU9Za63>63okVLI^Dvu~w330~K^2h9u^9k~kmryu3<&S9ed0nD zL^6p^px9s>@FvKB6+md=5hm}zyhSGFfI6|kWnfWpU4LnP`j5Bo&o*y={`vIf+2;MP zZ%%mdDUA4WA8E~-ffOmdOiaT_6G!y=d&-C`5gP6d@$h-SQTkC)4C4t9 zLFzK9okd6UP2(?4(^>b2YX_Q#qelaF2gablqNolcKw1~Q3=O{4H2!ps4t%GoO+yd{ zv?vm$;Rx@?F>fIAp~m1J^a;fQDW#*uNnwq06iGV!meZPPkgaHuF?@Y0($dUT2ETwb z?^ZNfCAq&c>>Jwf78}R>!K-tKN#31bL>}h!?9pxX096JzN#3{*Jv$k^Sq0E2Sv=2o zROd#i94b#{KksVlg7Nx~PWr%#QlDR4Xk!NjMW{W{lWzukW+?wD2tE1~{c@z#@DO?T>9l^YeF?61M}szkB`9i9O>v9kDZwP4 z1xcy`U@FYLunV}M#J~n%&SA>$)tc`4gc&&RNbVD{0thGW@HcqmSqzkz9KBHE54Z|` zkR!>CtTMgH|5B~%jmdEh8xg$|^cgM1P#(ach}k6~a&K)9rS*|}v@a?;-@skuDd|Wg zCH-gq-bn*aQ~~v+JZOs}-Pp%D^4cp6do>9nRG=$~xpUx1@dw6`luMqAdBjGABH~bT z_9G{p1Xz!(E_%`Equ8*GU=amiTHGK5S2-^B_V2IX{)BGv=}RC|r|5n>|NBcm0*vJ7 zQSAhUx7||tw;@~*FLKsD?J!F?lWOP#kj@2>JG$Gc9T5lIQDz#7 z0{VDi+OeDA9cY@8N2a4z82=K~no&LVW3=|&g*rgMj^>;Q-=?S%sM!^|ST9o$=d})9 z5`I>~%}fAD%EW<&3J(FPIU7G~q1d8k>mMRmEcVC3Ukdimmm%MeH&TZDF`yQCXtKvC z9K0DwDNGfBc{ktD2`*^s^k=5&e0xqT^{v%!z9Fr{)ev_AoEES>D%mj2*y4Dk*+BGx zqq~7`f*_sZuMHxf+->!LynXvmZpQu)@D$p>WX&u8b5oba=(hR3_&JxK>Qmvhf%qT+ z1Bg(tR0|l04A71M7dZzWI`;$Vr-?(LfWRD(pFtqAhd`d(Oz$3W>Y2=Rg^{{}XN0f} z*RKkfEN(H^x`AMv;|yO-b$9v^$+sjYBTYAr2#T^oqe-{Qvr*F?-)A{+Q4k5F z`nN)I>&CE;R#7hHGgPPgOPV7p@d?fjk+f8vOL#MY?Pt9t#c7q8Tze?_kR7MdB%gTq z368tlvHP^SKL7Ahwjns%0xCh$-~d}^_j$+}LRkd(IeVm`Q89nmbN=s7k_^*fDs2vn^h4kzm@_xk2sIGTzG_$%Pf$E!towzQi8glJ`TRAFd zcW~FFj>05Vc^^y)a#+QCjYi`6jS)%6n@M~3Vx@ zju*l$IQ`r^O>N~pJTH#lg*FeuURu9tDC}s;wC`gx${x>V9Wnfs`W`e#oqq;iKcFKn zQkUY-(d7fu4j4bke8{ZaP?JEdikrDzYjdjSItMnvCk-P4E2c!VB}+c`;AUW+S$)0x zqgPbX!Qs=BMdGzaM>>8rak`c#v~#*F6=>ibfd<^3?*u;KPE=%#N#p={M$ZvgkBvId z;vUE;ojjTWGo>>E`J0rsma5q`^Oqovy!10H!dWIXCs?Q9Yj=%;?F;!u7U$$2zb_C9 zj1Zm#S#KEoa3tJ2$ii?S?~(#Pjo9^T`Q@LZ_LFH1ypVJr3xdW&6gc7hL(-Be;bIo@ zWp--vKJI0I2-d-;Vn9PXx4m`T3isX zH~VWq>ckM7{juR;^#DRSOBP z@C-m1@*G=&$G$D-tJ*UAm3#T4&NHmp!r3tpk>K(BOBT*^@Xx^okobei9FlLc+wK;T zFs$a2(lWH8{(O1OsZ^5myOEN&sQREol5+&m1RSo{K=Nxu z?ozqYzTyF5(-coOuF^qWmdT`%lS~4Mp2_`Kf-8WKd43^ERL(m-PDn!;R?JSoCKYBY zxE+x!gNa~rvm+JuiXK!x3pJw-p<#&h$dg>snYjJ*ZF3Eyr5yaYHQ=1;f|f5JW<@-7 zl#8#7=Xy`FbiC@GZ4-!(B~K3q3RHlzmXG`XFuo-$o=&*wN(`k9BA9g&$J6i>u@rkN=~)c%H98tF<``Dud*ch-uUvf*N`q5|HspcX2uu?1%9p6u~Zbds-7rNfGmU?R{vh+k2` zDW%3O!J6PmU@=F|AOe1p8_~}zcq&_-&X!9u>PNVORk>DNCVe5~)Gz-KknK;#obbqv zYFVJNgl55lZ(@H|@qN#5-tw1=7(KIKHZa7^fiOCc6Njd-8VC2M@)p{fFNF zj*kQ5)hP6>9Ylu%Mr=Mu_#X$V=nnyx&1c9!p25xDPi*jDWwPg29WhuA!r-DNKf|(= z#k-Ma(n69=llMB_qd&q}TOqD@)(|yK^LhM{DwaAu30SkLGXA!jcHX}czqnhSuW{HK(367eWyVW#> zFv($}@ZDXJ2EP3k7_@m}YPCIxh>UC|A2JMMRBmCuo%hoX|0bPv(SMWPQAoi3ABF5m zW`24YB2 z$z-VKKuf;wtfw_Huk(f$I~)%`mYj=HQstNP%d~j=>)E?s&-~Gtfxp;eTp}%J>lURPwTNWp<4qp-dJoZGeTuQWy zjU73UwZ8}5$oMN9K=L(|HMePzfZBu{+Fvm>AzH?#wr~xF2IOM@u;18DKl$6KSTdAF zO)is3s6oFWs6Zq|;6|^zea9b7k6e&iZ>cAMcY-I>-&NiB$U{YoLf|+TICjDrwPB(2 z#BIaTgggUKlayMdd*aB84xIIsrSt=5{pOx!@yn|>GJXcv5NW6p2q;wmH@;=F@PmIm zp#?<&97WItWWg$0B%<9Z-o*;%!2DeMsGPT=ugXe`-EQ#4LK_Bm!)Tq{n=yj%3eYN%=@vpT&;>SIGQvHbWu)jRIzkCFN+1U|zs?hy z_<%mZvgqHuR{a^X`M9=pmJrDx=lJXGt~vR5cY%(?B0X;1R#=*2G%}7lkh5YtGSP^o zvxxj{oyYGppN-+z*5p85tgsc$VD#=i)j5{sa>D<;^uCg74nzW=r`d?&7a(k@Gvl~Oew)L=Q9D#0LHuYgD%2(0t>?T%?^&K z%@=r}sJOyAR?$}j#4``qO`@QC8~xl@75yoyM)YJOm7Ih}cw9tCQIxy{z(ml@^r^a~ zaSyp*EBsGqLIvy|1%40AVG2GHp~77st6 z9&jO1kmjw?a}bUjN-DOirk(>hSE)p`oj9Et}d?_3`Bchkt~6Zoy3rTj2}d7 zzXgp+4+Q+WXmku)H*e$zZtehn;q&S5LbD@}N7<|wX+t3fBbIYl)aKyCJi-53S{nU$ zI+0Mn(4fdJ5(+bs5zxQ=`Lm_-3T;^rKfzu3q>_39KZWWP-GzIrDXS42sfu13QKX{~ zFNEO|%5lu#rIe;1GlTsnmAoqL`+kTgZBx>T;FzHYMz&)=l%04_Eh3ju5!rQRpqP@> zkWa!-hZ>E4V#`Nrr)(j3PfkZ=@CKGiM~D0kr7wlM!I=&e1sqZ%a`*oF$s4DM(=oGfFky7H#2fEuY~IO)0QCJNN6) z`l}oy9@Yw(D@zMNc%M?Z_II01^}T1nq}dxj^Ah+60=uvhRer$8(NH#6-m@gm5G6J- zUUcrvc7|lZRQPz#RhOo2%bZ{AE>ncsApCc1$@(`KHNsn5x*GW`sH7SJo)U0*zL5ZLSmk(*PGfgL>zqMotumetURL_grG_6>y5o45aZ1CZ_P`RPAT zH?LmEN~B6kDiG-BwoO5q+561nlDxZ-{;a)n3uOFsY_JQn0?h`0*`Y%vou+oqhfibi zivMIaj@<=NGim&?wqpt9@nKNR;Nc2X1Bu+>Gii2JxXU!J=Nd#ZoVQ8e{EK($O953G zHY0S(8)^K%;w#>#ADwInxeemaBji2l6ABU#?d_oBb@TJrJwoxz%d_V^2dcNkz{IUs zT>%%x>u$xWSC()9yQ)5S@G)KTb&_zLFjxPjjZF7GjpxyRO5d?{JLw_-5~JxCZF&0r z3)5MOsuL+~EaKQwA$QZMo5u)NS(rn1eVbO0m7jtL{riZ2CKQ&fL>n6ci878jbtKt9 zrR_?hL^sY4jPWF)H=@``P)k=<OXjxF7j`Ps=ocyLIdw6C&}1$3Lg;Fk6BU%e>{6`g{G@K({1Db|L?#^PcnABKlY4Y2-&N8IQA~ z8#RpvlUb&$v?V@kx0t-tKkUSDnX9JpHBdIDkB?pknzrK)6T?^|USI-aI8L)s1k6x% zbXT~LhRlbMfxrlYui^<=eRt!T`t|X=*o&?-`}?+}9p8>B3LQy3ny~(6`&8Q!UFCIK4rMvB@05No4Ry(Y@ByvRrfF=OWl5n)e- z!X9I?;VLWr1dk$`b)`1c`+BZ#pFOkh!OdKX49ZJdof0HZXx<6;3EKd5wneGb?z6O? zQGb9l)nZmAIJWPB%e#l|61Kfjgrew4-SI|l;?lz)t;~V6da2|{=^wsTOrqfVJ`V8K zpbiegjCeN;HuX83=&;ZH6QCQnEV@KJRF@EFQXqvN+`eEmvrEihY|Nd}cYigl(fx+0Mc<6GZQXmI{d#k75&iVxC8pP4~1C^BOZKy{U1js&SIX&ytOcC|+gQO_rNcqN! zH%eu!f5CVBqaPIjvFY(aFYY9jmSl*n<$1$Vj^vFW!@krCvF=nD9AGOL6|%ut@!7TH zB#aVchUjjjYZ69pF;+l6{EIK`oIXN|<3r@DC$uY~wc@0a&(T%|q>IVl(3yfpOcq89 z>x{5)v4izalJpmmShZ=NrBewqdEc<)e&VkcQS_Mwf2z9($eo&aFYc)b9d8l$M)xHm z<64zZ%qhG>m<$2QE*+2^uq8G>4)zt+SQjcw2!*dP@N{fk3!+hdIhG69E`;FlWr2#5 z+Vg`}b3b6B%9AGMqlhY^OPx~!Co9<0tOBYfsT)VKQ(5xM^Z&DX{r2yh*Drs2`Fiv7 ztJi0GIAN*j3d%=N;X5R!-6dXzUeHVGFrHj~`b5G6Kjpb#sj>OXGiWto)>G&JO$yu@ ze%{)TuWKpx*%p04?a*;v^Khgg(66y+mMoNZ*QUt{MPJ1P+|=C-Sp@|}6F=o50734A zM7F;wEZiUCxfea|I@m^<-{3Hr-gLJ+FBSY%oN#(G#c<$mw%>@h3RUXn&)r&wh3B3lo4vN*$T9y(*w!3Lm$!<88 z21E2%INDR`i{%vBRqiwt4@rVgV)%vaozL=j`I^hrnw%H&TOItM51iVHiY{u0wa|9$ z=()AGoq9u4a9`tr|Djq4>TmRz^OVN8(^BH}@ z{OKb!>6Aj$qS9FQAkA++-rdTPq)p~bY62NxLwN|{&}OWs&qs6H$P;2G>Qc~XxEb10Xru!B6`YFLsrNFiEfo9%4*R)aEU?|^NAOTcJW+M< zpZPKBS-=vFnQmwjYxlHRkwM;D3kU)y1 zr8$1&C^K~*1f1jzn-ug9ygyC9jYpA_8DXu3rFih3^0d6fArJ9LD%MSnST-E_6iWT0 z{8dz&_`Af&1^8Sr9O3Qv4JJ9m)NIucYQw0Q-h2leR2p5?sYa zo7TQwce1;<TTM%>chL|Z(f}QeJN4xgBJBbkHu@2sGKv!2mi22~H)rjd0!zu$@^$J`?9S%Cc! z`s~^Ju^%CsHoPEJ9T?x>DuLRE)}U|``8ZPO5Ggx!r2*gY9e4OI!jNQ&_xI zYT7Zae)6KKUY}0vfSlEw)_}fvmzH#hs74a`)=;*%cbNj~4q1dHO~P1=30gy>_E5wI zjb$sd{n6<7M&61vkGv4g-r~Uk7Tob4{|Hf}QklM5nmqUH#5aXw0}~e&xUmp;^+oTm zeX5U82&)zwtL{Xb&X4v`1Kysn3)O!$?ip~@yj?;4HHRrtT2sLx`3asF@H7XDtx!*8 z6irKGvEX}elq-c{SUaOAAJWMH&AT5ayeM5Ni$nG=%xBB)=YKpW?1Kc4^$0mx4GE`v z-RTr6P^wfJSFQjBPk{^?ACN&bKDaBwn^ns!joPEop}i4(&hRHvt?*ZCBZ2hUecrm~ zA{7OPHDo*>Xo2y>cPXU1x{&iOt>vuqS_68_fNlTAcTiWE9{q%EM^DUlplG%FZu_FT z>z%J%kdwML2T&-3rZ|yXbs z09u>~8dNLtKPi|f|Fn93@mwox*qrlVc4^`oq28dxX!ANvpnPYC+M|}zmL7ks2%QsP zH^Z&1gy=S<=8ic~e68W(4EnyP4wM`KDIE19SS{4dAOC{LE%xX(3<&)SR!(p|={N}g zZ6aQxAM%Hv1;DoWkP`91w7E@pjdT%{fASAtrXUKT8rG+#M==fEhs%owCie2|EtZ?0 zvCbgVfb@oz*tYUReCOpzIZoV~Ca-9x0<3Lwzc*P+=IAH5H#jTOGlpqI@nCrqQm+e( zFZ+S43g%3lrJXlq_Seh2%<2I#f6^%3;ge;)XmrBu#JtA;ldni84tY0vX|_pHmTq>Y zN5I}pPGOQR2UD30OJyo7jG5A30#_#p)QThDk9l}yII8|7kpL^COAE4Mq8i?hSs2PJ zpTOnM4%xmLi$+dqpXI=#gb}1Gi?ICinqK4-tBKkLx#41h)**$=}}Y zc9cc+voCI|$NIBpCnC}=KM~IPEfmP(ZVN{yNWRM7dIGa}93Y$RjmnI$_hE5&xqFwe z3*lRwCn$C13}Wv2%HnFQEot=z<>Nb1ObDRdP%Ok8i=TA@SsX5ta!@hgKJ8AFdEdvxnzL+I@5(ftU&HxvxLTv zG7$%tk$pLK1SIf$LiY(Cks#OKBA89sau<8atsk?T{~EjB!94{26cigjz4&V~l;owvz`93szx@5v<(ABh^GR3{aN$$b zBOgzhnO={0MaU|n81I4 zuuU?cseeAVdi|yf8Q5P~OF*4$Iq?C1SwG{=wOE9LFtx$8Nj!qyebXmyvcI$M-BLB% z!uQC15JU4f(dBDQg`7QaP5j0tiU6fWXBr?&-?uqEEjIaR|o> zNPH-faWS|4`MnY%$A=2Kc2sTv3dT>+h5CG7T&EPjJ0hb`q)mxaWcSqCoj)$fbSWQ^D|r$w*}#C5aYQHmJhf*b|p@T{eU+ zX4uQY4cC%K_7^^^Y~4zJkeEm510g^l>jQy5T@HxC`M0y~B4I&@Lx3YkbX|?xO1gJy z>cNzIoHO?$NSAj=PIstLQ5>mDuz~JYmj73WV@G=RJ)yvh^OeW9z82Ck4`8Ywm`zeR zxbw6lZ1cA=hl0i!b+p zI;4g|36CT>)-R*7@7_5#gRzp&!toPvy47(uZO83yT1JK6U!4 z4@jxZ(~A50u5Fc5jmhMB+zb)t(<&__bnrNXA+|-{+A9h2CW>|*1|!h-?5?l7Z#1%Y z6m*|G6G6e}@;9gY7jo;IDQ6XdYpsB2?}kETJv(>%bcq-cpn!z{ zG=h+$mjG3#$>jW5ha>6AYKJQf3>jhxKeqYyIa8Td^9o@K%8xpMVnxter?r^1meoke z!`=&Pu@H~AN1TSJhdGwea76PG!e0DlFz5d2kfdmKaJxx=OPYv@3v7-k4c4Gp@&uqK zh{||k_Ku7%dT)toh8u#Ku@~7LGEjJd6}xsM)8(%Xk<~$c+#1`wA!9Ea6(Q0A2MU}# z?w}tc28~Gi8WL4;UE!F5jIx6b>(jUe>nhjpq}vdhZowVki#fm7OonFdpD&X3wjFia zsj_{(B1Opy#+Q9dK3tQpz|U8^=raPWa5QN54O#Q#An|{HmiXkj%Ec_ z<}F0}YztXFLR9GO99UZmewG}JO%4FLh6M}77kAksV`V_Xy!95=FnDUhAbJ8s>pEcm zXk?|U)d2vds}kraOr3bXk5#)oT zJ|&|N&+O8{rho>2mB#hF3YS2{0#=2L5M5dEr2_LM8@D(n>{ZtN>O=n)43CgIndJe1 z=YIEN^v@2qIq&`a{~b^Jzu_sTaS&REe+ zr2?ewz();L0;>X5EiypRR(cDh!H=h13BSI#-rCsv z7Sb}rS1DHr*-iEI>44#;O17v8ka?vS!9R%lYzJJ6#2N_nd|jB<0YXCbnrOyADm;w0 z5)K7DmcsN{ZpJGQF9KJ&>sss&M=o#ao~boZgYzcwdLj}QoWa-~j(Y>8C@E11H#Z(!iQbDiF9$Ur_CjiJY^EIr-^IO`mA&Og8Kp2cNMB>!kg<$ zZuXb5!1>?4UvW#F0bC0*wRCNw9%A2jo{FZT7^}{?XZ!HZ+E!1cfp$Z6*Fr-mQ(KMG^?;h+2eL^mrcY_`mXU zxOF%g+bv~32#*RsIm?_(FhUDasAdZ?vrA}g1!)KwkIm0CR$>dyS!vg00;=s68qa(u z3<2=9yTQ!gd`SOuI+Fi`&qDl7IMUtqlUz%d&)VB%YvNGJ5=PawfX)iUV@q_9yUWeD zWU@=`3gzcGlR(Yo5JW*^qU+g^78jp zPz{7J4%14iY$k6R!U%3dMU~p;Z{7rBP>88#F8E;3>PqW2nQ)l1_Mgm^Jw2+tKKb?i z`lhriimIb^<2oI!c0F^gn?k@5Atv%QyWN_-cE9wCQA54RJ zKH&L9Ps~H>rVsXK2=}qecXUr>)U*n^pxs@bkf$0@)FvNH6$vu9Fpy{D2%Ke7uL8A3 zf zT-T%_={$S5c#ZtflC{riOt2OJpa;PF98-e!0&;qE?aF>k~CTT&xexVC)f z&mD=UU%D$XOt0`L4)lK5W%pEOMgE=XEl7aDvrqREp$BfUnVWEu2uezcomQ>o(?x!*V470#)N$x_Jv`kKE}n@$IhGObsHd%TWpEmTJEYwJQhWb#riGn* z;PipdlS~y59pB%`mlrYX6pClj07EHg{^eIWwMqy{fcU37sIWvO%T`Ne-Mfk!ZY>C4 zk-E5;a#)(s+(je+EW%&XTDDbp*|dZ>{alj54cP1_H2>0jM>hy0)>tv5zvI=-%1c+u z<WL2GrUT|>Yp2(X?Bq#ia~ZJb(%EV$OIg6yGISLS#Cvj5lTJ;$48eE8QTeCXG| z*k4;IX1^E{&dmX7sSI{-`Y-$&P4aB_*u>(5giqs}%lRJYuu?uiZZ;HkenTLyBz+gB2@v%clm=<+e>P1PjC>3;MR)l%Lectcf1-PSFjqx6~ zyNo74e-~$1=X2~rNz9F99-VTO%R`!7i(F0HdPn?a8q;XTprw|wKpSozcC%FES2Iad zTmSCz{O0n)cB(sBS5Ac0KArgs9s!j&7K7=nnULX22lK?gND7@9aP>VxE1z!T^O%pB zh`438lhgwsR6os5$=!L0w+Jrm$@gNPJ@(`7amXCZ4Gz&{E7QUkfR>a5PeuiH zBs=Ubolb`(@hOW2i_&z3?Vu&f9Iiq4+r+ZdC9NBlm*hkGK{cRiIXMQ*3so ziMfZfT42dZcBLsAS0fOd)lU|k4>Kkn*SVh(2@Ct{*UR5WSwa-=NivflfanQmyBidS zcFiHpJ*zp&p7WwpO2^2;&aa7u)WgqX6X!oSAKO#^;r{61x|5uNbQ~I8q(3)HU~eeA zY=E@W=W!{ffs}*j7$snU$u|3U`pauF15#I{h+uldrG*#JzJ6F~fHdX@$x3VpK+L}> zHV;cD{qdm9A^zXSBi^^);r4~kvK(uQ7r+qWOAi!?+`~-WocZDOz(#653c1t@(sycP zL)VkBw%Kr9rSXg3fRz??0l`dzkY}YxLg4XS0@NGFJjv0^gOaiKH>@nj6Nfnj1tc{c z#ba2Numb* zSYS$tHK>_Y8SraZ-u)X+ZMd)c_}pqaBYmHL(BMu<=1utZQ$;RzX~$>bb5xTdw2pkh`+o`y!MUSz2XvVl;1_wO^c5s}j{k@xxm+@Y0FTnU`)lBRvxYit5~kl`_+ z_6f0e^oSD(1?h(S32N6#fkd9Lg^`GO6{7UkEeg|~tf{H=m#mob*R!9S%?a&on}5A} zaVGjYDBX|1$!HYZqHf`@3&%GCyYu7Q?}WLdc*IIftpazY96$#%+P74@_jk43yrK?P zT*%X>Kf&AV8{Us5?JNAgMZ37~v{en;b8;z#R>Hzai7imMY^{yG?V;=_xr ztzF%RYZa?q-RJ5&y5qD48X_q!rcB{3%Z;p#ghP z5<fM|4zzWP&{0FV+WAsMkB|rBdN?y8({{gHJq-W1Ra<8W9rFez1N{j&eni_+triH zSbDjQJG@$U9SsyzCV&xA__3LY4g{*+S@RD*qyrInx8~I64)&?o4tyJZ`di{26sY`I z(|{>UD0UL{C<%mYam_!}k~_QpGScXMDS$Qky<0&Sh~Ucn7bW+RHTHMzA$r{rX6LvU{Hhk*Ws+>{xm8jJ=)JyAca%I#PmE%ah7Xn^fP~%da<<&U7!3B(Fb9XK_5?9w+>@ZJtcS4)~?W0>H*AyK_S)C zvqLtVOn>Sx4q-#K!fAWxD>^_p~@N)Gq9CS;mxMJ9q=Yk?UfwE&^i;k4;(9tH-Afj4;U@3;% zQmW1Lc%1OlXn|%?HBAF_I6@Yeg2_ZM$nI1;Ma8Zu3$nT(70JH?(DOsdw_n}#bdP5+ zXh;nMUVi`cQ=iGi!et1=-++wEHax=+o<_!|B(fqYOk)OUcCfbY733nZxle^h8N9+- z?UcxyI1uuKR7A<$J(dQl#itpB!|!*9BPE`6?StAmmzu5Gym)^0{JSvVAX!R*xh3I3 zAHIJmGk6EONY;P@C9uPW)|?$#{-pN9Oo6#Gba3*y`(qCcrji$0Dp;4 zr~@!@WT3seEk`Kl0U)|3IGTNYEQT6Z59k+Epbm+{B)Y{ug)5@d9)a`@B$2|qI8!t+$3_Dmk&>F! z)I@R0Y~BdHrMO`HBKsJEK;yf_Q7seJ5*{MCK0j}6K?hB`+EH;59EmI)$MobF7WEA2 zp`<39Bov2jHBQaSG!!eaHyV>P{kV-}H#;2u`1!&Hs;4*c0*uUV zi=-ApS8K~jPbrgn(be&9L^xa~k)C#K!4Q&^0b<9YL-D=aKEZj~8#wv+kObZ6>bJ+CMM$?n(GD zOeHNt`$yXUE_%2Z?#X5R>2n<$$kMGK-xU$a%N%m~Lp9^Ad`?Ja zEocwNyHuDEM6V#gDd2AKW4h?!5cb%+x{t>T5Kuy9Ks7W2T1M+xY-^lDd_+raJ&Q5% zhLeh{1DbRK3L%+3xdFxue{f;UY&k3sNy()j`nM<$TyhP_^i153?-30ii~|28)xcl< zu>T8YbV@XU&qkF15fPMT`Rs>xf0}7y`ND@WU(+pij& zDc>xm2}FsA7E0RjGJmCddaLm&X=DrniXO1z(@?os^-Irieo$L+aUww}a3UdegY^dD z4mxL}S68mNtL~F>&Cvvc4hL!=TB+GUlpqB@G)!nTX8@{-Tf)<5t50&|aLe~3I6sfW zIjH~3-Nt?==Nc(zBJgIU^|Xd8C>}zB-_;(Ij=WzF=eE=q#gDMtim{#nU_<|S8+3n4 zaq&ikd-(5WCoXpiU2q0YnOYtH^`q6+-0kMjGAs580Ww|Qo&(q);-k_ZIQWNReOAyL zwDr>(K&6H@gTmj&Jp232;;O{tw1)~jCgNX0f)8&uX-u1yY2=}*NUox>f zLTOTsA4zl5v&lt49G;Ep9~owBEvuP(e2k3*JQhXV>wd;m}aN+W(1+cWo(G&JPQ6($pkmcrVAt|-a)6y~f9Mc!y5 zN|h2}q{Wdqw+_Avje|CIk zz+S*o)m&1{+F9sDKjIEJ(m+KAB$U6}?7Z@DD2393H1N3W0J;ieQI~w|N~Db3_l7?= zjs_s#5HvjMjI^~&50?)B0(yAq>W6C#ZTyZ%dJI3#hjuQ1__6uW@7zto^uhGP5HYHF zkWwR5%tW(gpZaH?+NncGsxMI!OeI(d5BA%(} z>QJZXA;xjW^SCcBA+rmxQKf1V;F1ttHX4 z1DFU71<6v;O(c`)GQmR$OLv<~J8%m8*FUFj18)xkRF#=m!@2@$pt+xvDOMF!?{t*7 zR-{H_Un_##i&Q;rA3e~N)?XHqQxJg7lL+?aR?z?T;-=dL{k*xwIMSVCb`$k#+EAhW z6)w@n-`wj5EEz#1PyhJf<^L`3Ex@B#wy06u-Ccu2m>wS-k^lh`Vz>;W$v})42?Td{ zcXv6sySuv{+&ys6x2n6VGc!G*1NWT&f8Treeuso+cXf4j)vjGz)so^cq z2v6asJ!hEdgwz%-g=bJlL>izcp)63UfJ$kUo+%YT3Z)JF&H;||+zWwqwm=0NBBMt7 z3093l2fR3Ldh(h0tlDGAc?EmN4lf6K8x_nf$_l0v2Ne4IU^-SzIp8}fVe!G&MLQs} zR+F)6$bJOF7ExGW#+migj~T@evmRy^8H|raNv~~Mv(I%FW<_@O*rETw>vs) ze!I*E5Q_j-(pZ#;f6IkNokFxynnD5WUMuLTkOZv+E&%^x!Xj`%JWe-E$J9%1R?AR& zq(Bll+|2;#)X>&`Kifv5(E<4ase(v$l;cZrGVX%^|4mo;?vH&v!o(&SIw0n$$ov7x(ykOHK zeMSW5r7{8nOHSa@m0E^zXazcpFalOUz#EEOGYmovy61Su%_}Mw-bim?8hevN7HcW}$(|0CVgT)j0r0qh zf2B|4)P9N|pOO84=i_@JYC%3f0f=LREo31o{$WJjfX1?k5MdR$oewO$Lewf-X5=Bs1p_o|;$$P-8WOdX4r;sukh)Jeu)3~z^+;}j79Mi;GRhyYIyhBh zhi{UaK{A71LkANMq=h>8x#$$(lBKv~0~nQA^h?3UMF}!+v%q745?dIR{5{Uo7G^Q3 zayim+jR{L8JT?+@l2HNv4L)HDFey-+Yj-p8+p?=h%>m`8SO<=p12AIbvEeYayXo1B zXa`FUu+7-#RbUYo+WD;wn?L0WIjPP^yA(Z3VrP(C&b!HNR#a94jVO}DRaiYm7NHh- zomMkA+Hv6LrXgQU!YxANI?z;T50s|4NB~IreRCgT8>uff%3!_ z{KE}nM3DrlfdFDiZ%O{vE^1`CZ!L~Orfyqph+Wo;-(-mvr15$WH^=+gx5PH=6&1zR zSo3-u0Lwso)@XoBigT92{#!T%YuKXBWd2izOGnldq$52Ks)=cYYD=750rq3CqI5a+ zt+8Av+GH*i1`?FA1i%Z`g&oo z33?&UqZ$;|fxwU6$1PQSRF^SEIn)-8vH(fJG#BoXIy5N%-+5@j8H!`fBU{05oEn9L z1BHLLvZqHRR2!{I9TEkV76UMbxzEQS?JhcoQ2c}$!h=IC@;I3QwJhUMp2wgume>Q{MQzGE34dXpy8nB0u~XO2SC8eMByM%Av8MZOnM6s=_Ht+ zf8tGZsYJ*=jD#N)-J`e^2*=!js5wo=DnYK!HdXbxfw4xS@fFdrM!Y=X8-Se&GY>`a z08r+d=8xNK)2!#13tSRxba!_?;K#)_{YmsstaE=+J%D--4JHQ?4pF}h-f{lxCPO^2 zIc^Gh!x%V3jVMkK@Gt`~H(RnU8+OZLVu?kd2mdVVSx1yd14wKzJQ8MKpbkUra_&4+%SU*e}C7`slFdTt2P1_KZ5*u!J!WiM~2fG=11#ka=wZE{hf<6f&VrGH#BkB`Q{-F&ZI~(il9Lmd2&BJ3|!c^=v}PG?>EXTfV}}W zEHDcbd}dIW+CQ8I3{#4>{uj6v1jEh=w*s)WW+cVJ_J>^qdp~)u^0}9y_&TzpEtsvu z)uM;iC*|7Vq>`R1cOhv-1~uYvMhkL_%!mijHdAjTf!i9COn3_7{&JxE{d(9?LSVH- zsXWP_T3fW<5*8VgKxPxLLJ*NAQA-$mv=uU&_&1X?hB&z0iIIn#4%DraNfS*#-oYw| zpb99+jDY4d(OU)e;L>J6(Ff3rQ2B_0LU29sop!p1FC2#!!BdbF3e-Xacp5Ys6G{%a z*VJ+0Cw9e30kV)RBAW>wmlR2HIDM7uc;Yn_oPGwVC>2Tl(g}~2G@J?bq(+8ikD_Sc zz1U3_!SA6Y+ZgBdP_ha4BtU_eg$)2~T14n&8iH$NKRhlo?qC3FVf5UI5oaaofVA0Ks`fZ z55nK{{{NOOR13~XObaB~;(@|51^o|im2;e*ohpJKG6fD8gs7r%lEaLN4k2nBql1ZO zCTAFI?4X&ZZ^u0e-W<6T!wN33HzIw{7DmpHIFg1D|CoiWc?Alvkc9!FD6BZA#^6JB zu^+Hs0gg~BK)_7|iWhkqv7y4j8Et|h7)MiN@K_Kz0>`>oH*vWrNb`~$slD7Iis+RX%oV{aDn8*4=}gk+82?q6Na@slNlM<3X=c}M<&j1^<<7f zahkD*!EvRupa_Qh1h6dsBjpQ-!i4hH28GWut~I^xnSSL1bQ{B+g3cjWo!8 z#1u`Qmu%WDrHIyg`L@=#Xz4-90SVcF><<|tL=rnw+1~aP>2jro z6uXhV2Oe-aZJJM^bvzZiyN6nW4JI&5Q({~3;Ua|{1DuPf21IrvJm$!N=3n5qT-9n~ zmYR00c78?~o$Wg$^c4L=ONSlG=OZ z?t!ub8C`^MNVr*8PfBq<6-Qse1q0yfCM=^~@g@uFn&ed~oJr^nD$-Q&@fd|3VZo7^ z0e2>w&#i0>`xF@fmT*1E{bfKYXMkS_eNiKb#=izV&?rg7faJw+F$6io#}F*>rce~r z0E~cU!66X@m<_B~5cU_}OfrXvPzuKqVzUKKc(}@BgzAt1EI+l;RNdog&3(e21 z#6MU1DXbS{IRObzLsC!SJ)w_bQg+kGxl;|!&sGa@IOy35S2lQu&xEa2ON?o_5e!-l zZBCf2!1{4%6|hPkWdxLWhuaJIqaWF9xRW1e56FxNMuT2f3kwf!WqxENjY2h;#^4t) zn2aDaRMOji@um##kRqXKyc!`yl*^!R3HS<9MYs?0n_m_Ar^vJc7Y9}Vd}9ucq{h+% zKpt}IvFN7Xs%-8<1PDi-&~)`ae1Z4RW+>pzZ>x1N@M1Wm*)pC7w~pe)C7XJJuL&f`^QS z&W(UqkN=zfewp$ecJf$SDoY&B%6Jnkj-&vCL;g78$1?hSVG*mO>jcC>TQrNK2_c|H zU>4bC!0oYLK&hGg8lYsEQS%7h42KxK-;woF9UK~AG=$1!To4M$qTB_2G$=<%ALK}n zg^{iWQISq)MVzC3NVl}ABhk(1Nx&4@YUKL^HHlY;;pgyiz0F(xqlZ=uD z8Pc<-9vGZ*>6@-B-`2TK_$2Z*)~ zQ#U(1iV@cDrzFR~T*lcFOQ zlk=x1K({0xF$z%%ipS$DlUzu^NmHD|Alyq1bqWo06x%L@r-IbFYAZ z1NTS{@MD<@AZsw#Kw{!T8#dzLL8wniqcfncREMKJJVMwzxY0{K6>*pk>HteL#TjGW zE^-x1)Iw{|39BmR42!c14GT65;^+m+u?hJTw9)?>Ti3)Olhz2gC~RGXE5+}w<;+iS zGato3c@f}WtR|QTpkAjp@TlCsa=)c+#{@_js=t6AWkgjaYRa4^COm=8hZ2fkW^XZ4 zo@4rNDPo%m9wGj2p%RFzkK{pasn<(X@Zq$`brx`|>fo_dpe&f7y(jzXe?lqBD7g?f zhr_|zOsI%~<%OIe5V^n)VD~*;yDN>P7E)o>A{GmzN;NOf+gJol0k(s^vE$+k^YBS1 zg|mP?kubCST6(Y_fE)o8(YQd|BYKifhvBRVdtEWWLp_k#Db;F zC<#zlC~V(QeF&-v&>Xn706K}92c!`q;n9A=*ab#KnI~$3sE9pYL-GPZ^TPGQ@qp;s zh`^yzsBXgZHK^Ofw?+*g&HxB5*iBgb7=(t+p|%mT76j6$qmt>2W}HDB@8C*%#p$=4 z?iJ@0I`2!|1}RjtV{@c4mxBKM)`sbN`=E0kuYwdnoIOw<38*x9{`r@8INj^m6C9Zi zI(CWaVdmma91@KP`hyV_`C^Er*$o>z2=JphuXAfGfrE->sX*c4XaB}=48bZVUS`(C z%MFz{C66BnEEV8nSy=o{nh|W5J}lM(v>=eaHNSzX7>!y79vht!fk|Khxtc%D;Gj5W zAmPc1sFB7*aD(jM8E2DZKn6xr}D&B{!0s zGBCLyJ<9}tC*MP!s+?0uT*Y*w2&C}#z!9IDe<_$mqG*g7P@!6&d7xC=tUC^nM+OE#SO|&=U>Gybte+IzoaViLitpxNERPQ>bnKMfw;r z$AC=@vI^p=SJ9{cfvX;=zwn|E3S)%=g$*v}kN};>02d~K6e8R4(aJ$8j^uNI17VfR zXakBAV>@y+;j%K@P+bKe^H?jALG-kY8^ek^Xe}6~p!bmbYL)Z-;SN#I=WN1P%IMxM z0bp#=n7(9-;d3f*C_+>aejhpJmc{WABm$m-i(Uxa<5X>kq$rB$G8P|LXpI)c-0+)Y zdwz3&?M>}tN?xHMmP1TVDO_N^o8Ig=*e}cB;f96#9=fwY7h|a2mhF< z8;>Xpk_)hH608md*wh&Q(GJ@f2g$_oqcSWwyOAg&E9M2^2L>f5Vf#Y5DZO3L7|Eo= zGIrMvij73A0LLValMaIzk{ot6po2{s$D5%GTv*K5B2l~>L^-@ll2RTWp{T)`?-~xPT)7aM>C-QPfaW^kx(| zvKECyAGp5gNooyA&VjcK>9}nFd~qp|y<&gbl)TcABL0IZuf<>;MCDdmz&Tcf-nAxIts=!QlG z6GR5WaDJqP8Ng)$I%M<#{1D!55E7&QOb6h6Zp!`Y4h!r?D+v|jXhvM*+#UAYoQALT z1>gp1km#z8mNU*JN$hUV+hJN2gJ2gTj)0rhf-)_|xvwPY+KkSTB#=c|M=}7SC>-D_ z<>Drh34AWKNeg{xWRPmW$b{Cp*CZzi7wiBm`6)L=#7E3#Olep7=ySNBsH_i2s&7Vq_oI}2E!ixiuec-#AOtLgjEwkk%(5Y)MzY?8n&1J zZrfMZDpj(9*{+D~{?0C?pf#il8je zKWFFBFHSi?P9Pu{e6BexW||(sZ;mRYrok})2u6Yx;P|lr*?)ozNP@Eo5NL4m8Xbm% z#xf;m4spCdQ=p&d9;O5L<^Qt+V66!Ud9q%>MMob-&!R4KyTrt-76sz@a6y2Z9p%J) zhh3~ULBsJi7}1tUwiXEU5eH&$eyc1pjT~gVw2l7^16hZ}#?iWag?5uZpxA#f%mwp{JwL79aB0JIuCz-Lx2{7goEDKeR@Kr}PTfVoCz zo7j3K*z@)YM;dI(I_w-sdq>>~z41R!a$!(m$~nYv?B;!D@ZTW0&;YkTP^IKx?B?I% zmot#fE<{~mbWBtPDdWJwNn-)b6`c3*tr++w9DS<)C3-x%U_|f?8Fnh9$%7Y^c7ao2 zF0_gRCKF@}G^iB;AYjx1b_I#d zj*X^T6@h#m+&ds}*Q!lO)3Wh!}8bXj680SUf+P(v#4nky8*GQ%3eaGA{YU6kZDjnkYO) z;xe!gj9@Ra*HaEL2B0sJWF>tJIb-U%GbVn7gUpN_9l09(D{y67%!IX;Ylf~dHeX3g zigF0_38N-8IgKLX1@|C>P{rU}ujR*wn~KD#rbb05iei8R%>IN^3Ql>9E4%@qdIYE&s3?sF zdLP$chi{EUGZJ{j#*Je>)+X3MwT|)*iIYxv5pwXb0?-5HPsms>>*H(@;Bp~2bLau+ zwvgg#2229-2xt#@$1*=Sc1H?!t3xg&>XF$XoG1B>n>Bv|14ejYQ3Q@)22L`<)7kg8 zKUis!{c?o*f0+(NYG|k ze?43$sIdkVB2sS=r?>A1QK!jXR)+Pm2F66invn8M7CQOt7Qg_2zYy3uav=0N+>K>` z+AQa@WvX0Cd@P%{#NjJ7f@oD~!l?}e5d1@U>|@S##iC2|fo|3h4a|?<^OGw8q7Bw( z6>^)w0Zkv0y4j!wS2Yr%fg1{#Vq`1Ws~UC`#0i2#kqbeDKM6eW_i#YvUh!Cm=0+9+ z>+xk>eJpmNii5EuCxfymBEaRbLef*H&Rd|9FZn`DZJ1hoS>*OQ;5mz*4n z%0uDBC?F-x2_Xk3ySxyGt|7n1I~+k=(iD{!%l?ZHd6?Eb+{^C_v2zWhxQkpBBgvt` zh+u?TBm9Vrw?Gz`FMz$6!aF9z>|qBdyScB$793)Xi1LfFMMS~-|LG{Kf2qb1OpSyeKmJ0nyi z7h2+M3wt;xRspIaZE+DXK)7Y;P!u@RuqAkEdVV-cv6Wm>)z7-#a+M@U{a5VD*q`(DP_pEtI7)p&=MC}O$ zrqCNseGq#9t`KKU$SthYC0}V-2SsYJ&dh28hHV6gJJRP+D$MC4*7PDp1Qfatq&%b! z(_}Ge;Tc5lVeBCQ>!iH^UYF{wr;<>L1jvvwN+M*?8xBNLoYLkL3TLsTilQo&Ev7i+ zW^-HIUY#}-!)QMRe+2^D!9sSxRcNEx<>E}lX@i!@$`};1G0OoEC{H42tSO-3EqU6TMNG`glj@na|n)q}&?6kRO5g5l-HWZukI@1e1gnMKmTx zXz{;Ab4Lc53E6oj9Qt%{kl1}hb{y3l+yx7+!NO`J`c2?Y zLi8B?XE<5IuOJsX@_+8jP=zQrgU+7Ns}-V05wJ zWR*E-UB0yHGuSML;(S31>`mUCc^h=BgMef5q+o?$hUT)|F zGh5jvpp4(MmTJ7*79LN;`lz1 zYIFkqXNJ`bv!4#Y?=!#PxGo147Em;B96&ui_i}0ty9>|^b~rLfX{;HMYwl2+LSil} z*9J#2HX!IwS%9i9lN<cOC_byIr=TOHuaR3OU%?N&-uVZw(quLg@R7appgHU*?M75n2F9Rxh*QSo z;3Y%0KVSuLz6Wk;2+C*~C7dp#5DN*)`~!rE3Foh~*~#l3fxV5M=z%OiP_$r&vSSj+ z{nh?3&mQ|i%*dLRgCrG3kQS*U{1b(Zp0!b2uwzj|GV~cd1}u6pLGUpEa>H)f(`ciWs#xJ}KEmU;>YP)x0@-YTk~b)x7wn|Rp*J$r zz~VsC)MNk?BDiBzp}rDXlrp=%5(!tycZjP1ddGr9Qwt(${2l+T*^ZoHEkI9Dz6Xaa zYRc?;NzUC`v00LeLm*rJKvF0%VAR>=a7_JA`Hs2S28a=ysT6Vp3@+4=@^8j%C8;r0 z0Qw89kHkg`@DPOi|DcipWO*9oK^WjtKxGB(h_KKYgtVMjnj!f-4S0L#-hkxl-hX`@ zx#0qT2!P#Hs4D?*D6qY`iJ;~P;^{s>a@AQbAOF}0ekbrC3NWQb;e=3uts2F9s0Dz} z9I3thGZ1V@3ARi@>#qZY8U}g&8&NHU50S-g)L|M)pXV z+nrAdx36<`SeD3Uh_b|i$A|rCVW{|AVXfV;2|qHI#^zDN(Ht zJZ3_;WIuS6i%rF`OeF94T5?1{5MoZi{xcv)*uEt;)(ERoq>x;^ zF}q`?Y*CY5hz~8C85ti)!qxQU`02(V?-3z>u!CylMr49;v%^t4B$qQXuts3F0skT@ zn$Z%FLN8Cfr+1)V!yu2QUIBr4PnkuFcoOiT7^J%btCnj}us7H+3jl4xt+rrBPH5}^ z3knO@5Xvwdnqz zAm6??Ury;T^hh7XNa^4q1nda`j6mfwZOoPXRqbIQs40F>e_Tw4!}@@K4PO<#LEaFm zoj{IZr=FCgq*5W{a#IYUBpW(`6qm!YOh8%^DYRq}5i9^@fQ5--Tr=!zhbG9w7pQkMC;+GTk+ZU+kCXb~1;;#VD>e>`ku$8x zPQEG+C>0SJ8%9pjVC_Ym9nhl;^8EljZTERNwI#Nrgf|4jHmQBq3oH$oyj;wds3}2= z@Y+S-KnO5A{3QQGA~N{S?4o7k!m(PRK!fTp(l_A90Bi^Jl>oRvW(C)`^jRH)KcIfFtf85F$Da;#O< z9RQUERrp+wxK(CH{j9UAMAIiTK*Q6Fl6Am<$&nf(+|52YCC{8e3GOUJGvKTy9JNm0 z{7=0nNVc)67?Ny*a!w^}_BSI6C?y5|jlzt~aI*q18}?&CPv{z_Dsx6lKCojE06q!O z+O-M<8*y0CqLzwp$~mLdo)Se9-Oa`(0dXf>E*l7LMg&L@W-x*S*Q#|GEf-(*x4d&| z5T%2#0#16f$%1+^{%u+H#SI(-eu{8-z&Z3cJ$szTyP-}*gS-$0=+_Xtqc?INoegf& z+-LSLoI{i3{=s)nIAzfS3WrRn_Ctykin?L7J57S$IHC&9J&)uy&-PQycVz(kFN#KR z8V6sjO7u_{aNv#`Cflq%j}{pUBitU?W01xRMJ8%fEjE8y&BSF3@Y!lyQKd(SEz}$+ zs;mSgrWNR_r1}fkunq(DzkYu?<`XFa#_k4aUfQ5rC;1S?0MQnton2?Wa z1jH=7Tsl%C23IEcaZ_mo$dJ4pj_OWqLud*VIB*q%bAc>M6uCIf6I9ctwEH?9mCOm@ ztiIf@_ho>IT9f zwA(`fdZ#i#F{@!%s+F|+DUGn2E0SY029%IH6cv$)25SiL7f7%-8B{`3f8e_S0wapu zN$S4_OnS6W!r>FfK}tg^HfY$n{4bstpoI)NK>8?gSU?Jl^XL9=)B;G)2&JP6N)1-C z!$a-tA`t=<%KcErtwU-#>`FlCb5qkXPm`fMZ!i(@4b|}dl2S0FI3V1sc6dGb583=% z1?#0JSXXuvmi0%v27GWx!9`INV1=O7jSA$w(+4>8N%DBlSL?#h31702fsQZ&?t<^@ zZ|cm+PWiVM`c`uNltP4o{Xcyb2& z0RA4B;Zc@E64ZsKI3A9bVYX;EhB>!B&P#@{X#RsMG_Z5!MzFpj#TmAwovO9Bio~#@LYRq>aaO$v>2YL30l_N(+kx_d zggUt3?Yc&e2>_-MsVrA#W(pJvPX*|&e|a9`PWu#+lN_>L$)^~-r*(9{ixi4!$etOH ziHHaevbt-7WS1DgglY}CRFlNASL!q7uv>R^$eEpR#yZqt+)Y~+?`WWiZb-GZtr zvRA-rz!nV(c3}Kj6nsbikCn!Nc@;rjEuzO@w4+bw`8(`}9Th9V$Xi6g3b&%X&GD@H z<9dCw>GI^?`FNU2hI|p2Ax5M@S>OldCJBwgvk{%tSBU{G4OCEals1~wN|a7GHU=X# z#oS$#gaXPIwR3Wmrt8cadhZ_?SE#;6We^D4QILiF5WW+3CnF`PSWri^@k>+!*sMtV zjqVQCBGi9Gp%xCx4mc{L{sSl(@ZUK!K@0?qH8h5BUTQIy;1UQ_zGCYGE_T|TsM$C5w&Il1RM~PkZK8BCWdb| z*_1<`Yzlr*`vOMZHTr-5aK>-5$RvRmLILh`@WBXgMKusn8i*$RHatGkMPQ2&)^|d_ zCmJj4NzOyX9;93)2ys~od~adbX;Di9$T$>gfxMBM@{UuJ^L&dO;`USnAFu0|A0kwOPJiD_fpy!c-o6ezW1P)ZI!A5;_qH-TT`|1Mv*3H%5+ zF)0Acpaj|zeTaicmin_`g?9s>D@mJTtCcY_p|7KBlo zMsGmTraNIHQpkbSt(G%zp2GVKgBol>R(PMSI*r2J#oa1#g8(1l8-!0>0Z0h~!XpP+ zGD23sCB)7MhrSUx@+Nmd(h4I5`S^&5DX=dCzDKT5A_v0)TsrtQRR$vIJgiFbB?i~H zcadz+#oY>eGxC3d-Ue&YsKoIW`CPDoL7A%qiY>swRTg(aQt~l^B*|Tb)q%=?Bbc1P z8DRwdE>J{BLZgw9sY#p8ZzxBH#4`t1MS?_<{f3fV7bJ0(0pSl0GbZF`qP7L4@JMad z8D)qL!3|_|7bGQLQjk=e0Q;#0fi+ybGHkCZ3ljJVuqZfzK~AJjZqtP9SL{*Xm$Ax_ zt&I)d0QbAaf`n^;IBMiX()leJm&1Xp&Vo83$qok&|3;%4^p*^P_%eYZ32h@Om5`(} z28-2Qkdz!aK@vwkAX1^qpWvn<&L=0(j=<_vfMr&WH0tCwCBM1QrWWS~;2c=xO1RwN z62OA9$YGrTT~mc4pE9{k$$>+g&f@_nQ5V;6F_nWv1tq5p6_gwzDu@JBI1WuHo>3qT zLP6kP3~G>aW72@7Os91h1f`Y_71Ssc@X#aU8pJ$C{GTBwA#5i`1**`IoFcR+IpnlO zNwp{*T&=5hs}+3%or_B@uA$^3wOUP(l1Wem%IqDM>a`2ET8997MI>2Hf#GutE-CaCLA$~SyBmZbgwIE1JzP5|! z$nh8#3SQhHIgw;F8lXKuqNlb32VIRl1e+s}!&C~jT7i8E8pK_Ylo%Ny(QK zBvFe1{cbf7IzFVUka!#pB^n&x0ha~yI=P?Hn}&Ei7?%TwtIl&8psT4JI6+c!-~>r& z1y7$Yt+rkCT}JPg{6JJtQt?tj$(~a|NqtHMCErl|f*Mz|UnBa4I`Jiit7}@c$F33h zfRTuT02o5zfd7C+VT1t-j5ICyl}%QoJ0zVaiiX}SImOlDn^p8?X)Q@5B_D$}o%^U1 zUxF#ZzqDk7f~2&P6hB9S$A~o*t~&P-@jAXklFLYtl$vLNb(F%1 zuAy8lwfaMllzeeP5+x0UZ4mKD1F%j}b!k$9vIiM9IP@?u0Nn*i$srOXr4=uIBWcA; zB_)R)Pj?zi^SzWjQ&2(4c|Zjv)jJiG zd~GTysotreqNJ>5ilFqNFz++r&8Dc@4`%*jRf}~`J1WCz>3rXj$%UyOISDgjnUld)Fle<|v zMDh>@5E))nQdbC)()yjYDXrhBq~sWpr-xa>NP;e3hsuaw9G6^ta2JAdBAL3=j36ng z(F92u&iTL{Cs0H1%fOcfPad*|)K++!Q0J&{7bGQL8V!0|=6mOGc2Cd>uhj@+bsJ=NFf|d~sYA z2@=WFZUj_R@-2l%C5=HrQhLjvPnX^@sHCKF5T0%Zv9gH)LMuv?Q5y>EG>uXP>TGZ_ zAPd7HbBCn!&@p(7cyi&Yvml`EMb|Jew3m`IL6DSuA3;)5$J2*Peo8861yT~KE5JQ% z0fP#%-Hh0+5Y9#%#eiC1#L4)1B)J(-K|)E6I0E46+D^$*Grm zNtL)uO4=dGNhU~2ZwItZ$*v1+N?K)*bY8LwmtPTAodwlIGIgUaNJn^aKp zLsCJ>eS->0FMYm`k{b+d5lJC{5dt+7>JtH=57rSKhD#%}V2qGfU~s2`k~bcE>4EEld=sq^j2#xWAP7nq z9CHBHif&o3`+=(ksT^jKX=1h_8(9#P+yrSKB{xATD7gtzK}p>ro>B@|i$)<@AHb;< zT?5OR=$c$3zJ^^;d=V)7{49}Fose|ab1LyApmB@DKpT?Uhq$;~DQ*zUm~T*eSy4&p zql}`?_-GywwO8mjc6z%axs-#C6$yM51~!TFohr~IUa(f zCg_e#rrWq;r=TmtDeDXF*Uj^WBu*uBfDBhlDmIzmOma zm?x0dAvCH2w2H!r)GL5KS&;;0Kw`X1XLc7PCBJ2|Bxuu-yN=XN5FG&8!U8%i1*kb? zKuZL>t2Vh!=?$2^yYvQ3C8hm+R8n%}si5S*3xZOkNd+ahYAPtHd#Iq~R!s#Zw`wXV zxm8m^$(ca~CHHhHh(u9jXc8-e8fREuv{i7x?vQPe>NuBGxJF78%} z8w3gu-=L(@5hNwwPmq+hI>bFvphGUK4zVDJ5csZ3CZ!3Ik}oMpO1`8ZDLv`MeV4m@ zNpdx{ri8dnm=g4UNi8l&q9_}s^!bI z3j(xUbd7O!4RjX#n@jGIf~2&~LEDtJIjE%MM;D$hZAs8JCAF-}0dX;KG8t`Ba*(K? z zVFXFZ`2XDY>Nyl9IzANJ?%&f}~Z_Cqdsx+LEA>l8PHocUCWm zwYa#3i`;NZ4?HhOsA^FY1qK>a4>CrL5%q&w1za}BbVv0WD%p5JNxe!1B_}o&lr%Z0 zp!C<_`zWciXp3r8+rkw93LwH!N_e7xOs6uMP)cb8dYD0YYe`Q7ZBf#apn{S?GAbx( za!^4@lYOQ=>9~zX&zPz^n%jAd~!P zkX~7JM!bbeX8_g?a8#jdbRGENy;*We3zE{N5^Ym*38GDBU8WLWlDoKu*q!KBBsZt}$u4eJ(wilf zmLMtlf`X*9T|}QQZ5N5(T!EX#<^`@gk1j~a`N<;ry#-0h;SeMxheME*+z$mw$%P0> z=eLx*91dJ{76ed@=o+dy`5}_rHw8(_Ard4d=Yb$8SvxF9N)C}ADeVYyF&-`llMxH5 zU0lNkm+H|JBqf~%f}~_=w;(Aw;RH#^BeozZ`Go{Y>5WW0oPZM%%}v0qh^{fNucdm} zsi36sM+GG{2NjfD6;x2t-lu|+#vc`w)I?NJ()fd*zP&Gj`H|ZqRQO?w03=qcAiq{Y zdXI*FL9GJ3^sNHDJPFv70(?%O2?3n6(g>@=%RTw#WDZt40#{2oA!IUZynG1?9%%w> zZy@S26GjEO383V(39U6E)Eok85@oan`1)YH3)h3D0Wptg@IC<4+yV}HKd%5i!PLVK z-~KHk3Gfj($h3szMn2!KFUh$xkcYnVNIN?Nsas3fggDky2yQbEb{6BWcs z1gvZ_wMr)k&?o~qc)3+41LTGk@Mmh1h72_OGLm*O6_m7-si36!O9i!3<}VeLG=Hg} zq}HH-pyZZ91tnARs34egOa|lv0PP;!rhv>Ke7qJba3cX2q*coB2_63T z<#vv}Y0{J)hl^?Xe<(tJ;(9tLz*<{@c%Sv;_+Ww@^ARx=QO!{uPc)-O%Cp_ z5`SiIAP9uoj4nu8%n7*8%It4)Z=TzT8v}L~**Vm*P9I)mN8-zbwO1E^=(j0!!HS99 zTD9D;}!Pv!1UmwViNqSGDOQ#yl%=xWw1G!{hHQ*gJZ2tAY(j ze)k*Wui99G3tkW;;1^ zdC0L2Gpn~7wLI&ESXYG`Q2%_Azht@O(=;Ej*(;$<67R%w#w?Y6x?WRcxY?Wifs+bHvu?{nIY zd2yo0itZnWHr}626Vd1WF4^CC%RR7_t$%Dq$BheL$w&C*D$@Uk|KJ|0V%9D?>R)c} zYoEgR)M2|mEw1=%&HNb;7Z3UFzH9uPd&TM|jk)T3Zb!~~x9d%Kbz{unwpWH0DVey& zy?@f`=e`r#bXZpPXZ|j`M{jMH*5LoPb7IB3i@bK4a=y8&EUjr{E1^F%{dx3*?}xWO ztTAGImWkmzl7@IJ_VwLYr2ok|^0=AtczM0@f~wQ%o_SxTb>>dW3FXf0xV3xlgz;xq-#O3)&oMR^)2r{)un%1!|go?J#go>1%E?+spcX8RM3vah4*1FS5;v-lckT zr^|V@sn50Qe>WQX_3Y61cUsnMp7z=4O*4KPmt2{CE5q!-hxG$5zlwV=b&w8eejE&h1wncGv}X`8Q=SUBWGwyxP8o7&y3^_SbR825=G6I5BoJx|iz zOV_zr?dBy_6lgfO#Gpp{h8EvYyx$=2r!Nq?+*|fPrP_yux+c!PzFz@B?t%+^F zU9UCxh~MGLUbVN+o;GJ}re0_F4__TKV0rHBIlRZ699XVjF3aO7HD`=#cIT=-R9&do zIqkO=KdsYWcP%$><=jh8gFh8pZ1{esM1lN~Po`gc+*9u{b?MzsJ&q4iEf{4y|HCiR z$GfxayRCBOzuU@kT`xKAeunC2)jiA3?s3Mudt#|VnlvYFj!!67J1kSW+s0m*I-l?} zt;}gy$pZgukK!4<)kV%zqYM8 z;cLZ@0WA(D9!q+$ru&PRGFg4U%YNSZPws15b#Tdjr)6ChzRD1Fw*wbgm z7hU~L%KR0Y<(Bs-x_@KX`r}=ybo0NFSj633waxcyzcOcX?$1|3c6YaKqio0Z3;rR^ zhAz{@^`3L&bwc;qy?PhTyDRIP%wq@Snf_0orvIdIufO#1yjAI{te&NM{lr|Y<-=tM zs$AZ8tesprY;utsIZJ;ka`TCAwu8mDl}%GtX=bjiDG<0jO^XvNFFicj@M75?KRh=T z>wh;<@pyps`|iVS5?kh+HEmGVr!6W(#OB-lwQz9GPIrr>WzuyJ#tF9!>M`=I#e$D^}j3XjR~^lV6m+`%^^c2sM9d*p=XS+i8` znEArKQl`xLgGcmQ*lKfV+O$QyO9xeH+Uh{;>@|x9>YD}id)O|!ckuVMjh57F5ux0D zCr?RJ`1N-gIu8D8p?ThdQ}=Zp>ZF}f;IC^jChx10Oe5W1A77g%=dNtytq)cOJq$i! zT{h?F^tEq8?*w-HTK{y%K3nS-soHE@*>{cfMdKEq-h8o7j+=&0%7iIR(yqH7-1CTk z!&YO0i`3jYFQ9q4_a(X}RV|+*vE-HuW$GYd9p9@w_$ouuuyx$V8rw~B2)aY1a447Ed_ zoIn35^j?~_`wS~H)_!}UT9fa0?wn|^s{MIO^@3~D_R06`)`}``auxYjA$=3=o)UTQ z=da(T+JNbD_ma~W8#g?eb8}LKC3%XTe_qqO%I$-rCu}-t%~sC+_3#OwhYW~q?^Qod zKaX-3TF>iUZ(j|M`+0^>oe`P&_`vuVnYz>{<3D^M&F3R0GhVD@CMd#~%Og}!g1`d_s8t$bc=ShX?b*R7~E`u6#PCGx+ooc+tvigOpc zyRVkl+v0mSVej{MCsy7q_D`vup`N)aUAS>JCsu{ha=C z_0zJ;-`FmFo?N?Y+2=Wi?$PF~Sgl*(*Z$4gENYg^Q~gNUKV6H&AL;(Kobc%y`|O@& z!VXKV30wd1`lv}?DBNw-=(W|<5uWcF7aA~a?oq>)ZOVaPXHV!lxe1AqQ}FfC!jirDq-OYt z<@wLgBMVzB3>o$E(c|TNdxy`t>DP14&9{cIxf824{B~tyfiE6o*35}Bl<^9hRs2!L zFNdr5@RZkDaJy5M6&ovgj4fde)$W>_6jI=?u46iVn^f;+P22VGOy8ClcxhHw_I#b; znYOsf+^e+V<@cEb8s9#=E#cUxI26&A288g`PqIpKvn6UH3j|0H?a>R|KsXKQbntGv=W(?DR;QHDF zuhn}od zrtO=sGTqoQuQyFkviTehaqn8AOZer@xzeVu6&bni%KJzhN`IBS zuYX$i3VA=DJ`;NSS^bWAJbGImT^@U_f3A_!E-$-SpN^LKR0mM*}(B@?~-^`26%A75zwzVF$^>zYLbpUr6cSikGd7a@avUOepb$Xe-1 zqp)0SwUxC)j#Wq;Y#j1FU*!3AtuM^E(d7QyJAL04%lvNemK=A!Z+KDkM*0z_a%}yQ zE=xbZDf`z}3~%swQdp~Z%KG{_=U?ZX6n9sdrBk)oS$D^}70YnEK;`!HDwmf%Sa)Qk za)jlPY5lp1b23M0-yOUXqgp!n=*2f9XYT8AD>V1nww}XYCJYOx9K34W&Wx3H2|F7M z9kF2j!28v8#>G`G49QRN6YQ@GWy}Fu~ ztO%N%En&clt>bzY$X)htx4pf#Y{UB+R(`ZTXxUbnA9=S`n3UA*2e|DCX(ea6l7 zIh^-=iAD8`tebZ~Ter+Nn0CWXpLsW})0mvTAj@or!&r8y!vcc@9CFobjz{hkY8xYtwRdsE3$ZH``S;Z%xImn zzeN}HSa-g|^({5mwmo}xS;wDW{Lhcvv+I2IlRqB&R?Pe;AhKNh3fi@gH_kqqGj{!P zg|X114>`ZKtdPI#!+i}NpZ0GsyuwiBn}+QZdR~-|@9`=o`2Ds)UdP=hk6h%JnB}!! zuEb6)mOfk;stjys^}1ty>3;CRE!*JGioLgVb^pm&YbEruxRn^UEwpdYr<=C-0v>5Jvo*FI3eeY9$s?}`~annspe+J5FN+qP~Z+*W2`|vC6$_7h{HwCckQ1j{41y4e@fDGJ8xw#Y3BG%Bz9VzFt2Bto9)j{%~9LydZAU#_urqIMxL->8UqMjOdU^LI@07^qZ<7?oLoLL z^8A&~XAg`?ckgJ|=aatY->BTQ?fkXitsVC6IdH$q_aAp3?3x>5*gGs~U%f`HPUc+~ z|7p+6@LUP;D`Pi|>Xywj`@0XzuTOF-w{g#0lg+qcYr!{TFW2li%j2xM%*9IF*WC$`?RG@*`3DEPL$nmhu<-)x$tnOHZIkUwUnrpRT`=giZ<*hF@F5SMg zyv~oY8#>9(-z>8;VSsY^u4PTyF21~}RP`QRpY;q)o9E8#0{Zh0)Aa4R>}k19?o>~UcJ zUk_~qL$4(ji0~g9(e7B258Ibq&1HSl&N98;-bITvdy0=L-k@Ok$j^yg^9(Ros}}O^ z(XGqJWR`lpgYTB{pF3T9aqHd-**`XHqC4$9?0vm`tzYqhw+ z#gM}OXX874xpZhz#hD)cWy4N?x!Iu zrd{KyNqY|6p44V}d7pM0a}@5kIEVYWr*#%Z>6ctwnJcS*r}dS_4a(tJ#{8yqmnFlL zXX?BP8MAZip*5ODA&1jf4S17t&DDtDKJ`AQ3A*ERqJN$Z6*K(IJ-hXK(*9LvMU5 zw_|RT;U8`k>{6)7lE3ebys;(cgjK^5E;i709Q-iLuzmYahX?IF=-#e=ot-Na1A0fa zowDct>%u3;uTF1T#rS7kADqoYyXk4c{_6tQMrR$-bMMNbmp^s%mtA*jF|6&~&ON8p zyz;4Nqq{4r9ErNW@O1jdnR}*Px#wrw^1Ac6$G5oGB*Oe^_ta5qvaJ2Oq+RDF z=9%M;=IL_cAGh1#CiRu!%|pFot2Zh8{KTQ8J}utgKN;3KGJS@^Z#APU_zdb)KlgKE zNuT)lBf~mv-M{zx_3X30MGV^z-ytzgu~DUe);%=Fw!iXVpFy`WW?WtUiCf=sk)w(x z98VwoD9@YHgLZ%0Vvf2re$(OMLptp#vtdEaWwzPlVk=BovY~OWr@g~-4{SVl+=EwP zV=wof`Sw}B+^HddHw@I49A4$r-LvCn%vL{yS9*Cu%-+pS|&Uu7znX zKEE{7xAWSWhlW>OF!#pvRU7X!)mk5Es4MK|-*EX#<=Yh}4m5GUn>22asz#@!8~QI* zWnVF&_&*suhZHQPHmw;we}P-IvI&EW48CKHZa-p8cWsrj^N&oe8FeVl=sC|!o35{H zyC~#S?BenV_e9iI>RZWcc4$(3k>Z}erC?yUjJ3)Y{tzEJbM^qksGl#pzpEECF>kx< zpKIw8TNS)EaNmjehDBc=n>aJ{%FgmWhf2x|gc`Ev{1~%meBA8$Kb{AT2+g1FSv~)` zQ|}*63b{V|c#kdVACH~+yn9mY-J=&qkEppltheXEt`TI{ejcQSBBntNX56-!|g#Q|sOg zSI(X5mab*nj6V}MCiWbBeb>mQ!>ccq`u92RK1cgy-B)*a#$ZJr?aCrSb;r%xk}q;V zfn4bejvH9&{_8iR&)0naZu`i9Ax}Q;s4^yUMEIuKk&!EgFI=x!_A-lRea{l3#!Q)A za)3{!L$~wpmu;FjqVDsRnTmgYADS(_aeY)+nYY_&e79vO*11X2{Pdmw`fz`B)|qj! ztwMX>yO6ehT77t5l}C%6UAwnB61pUG8+3BqxP~vcRxZ-CVobKY@{1#LUbz?9_fz2=Q=YGIe|GW7=utO9 z4tIH8vBsE-AGYS)U+!aQ+dbo>hs}AFcyfA~c*XKT%(zf-ZfI2x|*IPDa-1LCp zgF#vPrtMYWwYK~c-v>-U*FmsO6o3Gmpe0dW8qR=gLXyEn;PBY zMV|fc9X{WiH+BD*WTUY~YT zwx!mVwPoRzc9*H2RONlKtOc$`mhn$}b4zoN50#3{jkC^r7+WW?!o%W6vz#5-r0l@W ztE-RPH#bMu&{C^5FWD1)z}%qMx*)|w_XCr!6?(t>)}-7=|BlYNZ_()UL5;dqJy`e4 z$c*QD2R`59)^YTG-HfqW`d|BA>Bc{A&-cn+=-tvTSr1IriBl%=oRH+JowUKx+}Ywk1r$cf`ON<6kHj8SiQcB@lr&zf@`-;UfrXzHzo;tRGb)Q;sI}qS zs8JulUK9P zoj&qVwYy5?j)cu$MjFPnUSsitii{AO$R zx?{fcl=<*)jA8wp?W=``!`B&4jXBUsfwV3t2 zYwbbhhT8((orxJadd04vFS~S{*Q$G$xjjFBxt{Z2g_%~jcg^l+-Lj`;VqmtU{(e`U zeXH6sQ_ou?7gn-;*#GQXWPnOu=kw>)D;__wRT=zrN{RCa)_tmH%-Qi}@x$jn>~C15 z&n)HIAqU#EX=*%HSebK)XHb*v*FW#s_9oLxeQf^Wmo5~aur`NBGmmfG$}EVwU4Bz$ z%cj@=WQc()d7As8FS`pQJQ&*OLA%*UM(xbHyOJw_f&Dx4`Pyk{ z^ZK_V%~Lkitv>I{p{0=#iJn*I8XonJth*}l(EMQy3{4xh3f`7A=5kb)bbDtmY!E^`&oL=l^eY^Z#J)=zUlUx!8@v_ zJJUXPMdoP<19N^Zk$3sr+m%*zn^nBrsNId{r@e5)IBQ$48ROz+UYZm;y~wP96{|Jo zva8w$wtt=_aO0#CUHTMxar9b)B|b%7XTH&Ne6@$w=a#@Z8*U$N3i~TQsuq>``ZGGoRlj{{5 z+?buv`Qe6n<3YTbCPJJg9uStVJK(&VI4Z2b<5}0Rz|X zXnKFM=Hj9*bqabfSQt7zmu7qTd)=Go!;h*bRGZi{*k74x&H65;x0NRQJ$n~2JLF#5 zzUrs1>+Ie8d1IM;`Aj+UHu!nHR-JabIBkWUCDz5Izja}J&o67+%wIcr?YnjLkIZj< z=jNRWYgUI3I~!dfgYopH*!jhWU29Y+Z}-19|0p~6(c-VgM)f&+x&G*{x&2MYdM@9- zOP<*+ONWiwE{484RWn%jap#jS<etx^R=wSPV zw;yhn9o4B^p0ZotHq*XIv&?;dhjAVg4o{5oDmS;<*lfibPk&sfLgJ3KLv{N*SM2Yz zwe2Hi{I)bN##Q_Lczs1(MgJ{Dnrwdh%q!=M*;z{LdiStI-u8FGM`XEEx6S7)esRsy VT+Bc6NQK5PCtv>jwMeb8{}0A3B?Note: These tests require the native library to be built and available. They will be skipped + * if the native library is not found. + */ +class IndexTest { + + private static final int DIMENSION = 128; + private static final int NUM_VECTORS = 1000; + private static final int K = 10; + private static final int MAX_DEGREE = 64; + private static final int BUILD_LIST_SIZE = 100; + private static final int SEARCH_LIST_SIZE = 100; + private static final int INDEX_TYPE_MEMORY = 0; + + @BeforeAll + static void checkNativeLibrary() { + if (!DiskAnn.isLibraryLoaded()) { + try { + DiskAnn.loadLibrary(); + } catch (DiskAnnException e) { + StringBuilder errorMsg = new StringBuilder("DiskANN native library not available."); + errorMsg.append("\nError: ").append(e.getMessage()); + if (e.getCause() != null) { + errorMsg.append("\nCause: ").append(e.getCause().getMessage()); + } + errorMsg.append( + "\n\nTo run DiskANN tests, ensure the paimon-diskann-jni JAR" + + " with native libraries is available in the classpath."); + Assumptions.assumeTrue(false, errorMsg.toString()); + } + } + } + + @Test + void testBasicOperations() { + try (Index index = createIndex(MetricType.L2)) { + assertEquals(DIMENSION, index.getDimension()); + assertEquals(0, index.getCount()); + assertEquals(MetricType.L2, index.getMetricType()); + + // Add vectors with IDs + addVectorsWithIds(index, NUM_VECTORS, DIMENSION, 0); + assertEquals(NUM_VECTORS, index.getCount()); + + // Build the index + index.build(BUILD_LIST_SIZE); + + // Search + float[] queryVectors = createQueryVectors(1, DIMENSION); + float[] distances = new float[K]; + long[] labels = new long[K]; + + index.search(1, queryVectors, K, SEARCH_LIST_SIZE, distances, labels); + + // Verify labels are in valid range + for (int i = 0; i < K; i++) { + assertTrue( + labels[i] >= 0 && labels[i] < NUM_VECTORS, + "Label " + labels[i] + " out of range"); + } + + // Verify distances are non-negative for L2 + for (int i = 0; i < K; i++) { + assertTrue(distances[i] >= 0, "Distance should be non-negative for L2"); + } + } + } + + @Test + void testCustomIds() { + try (Index index = createIndex(MetricType.L2)) { + ByteBuffer vectorBuffer = createVectorBuffer(NUM_VECTORS, DIMENSION); + ByteBuffer idBuffer = Index.allocateIdBuffer(NUM_VECTORS); + LongBuffer longView = idBuffer.asLongBuffer(); + for (int i = 0; i < NUM_VECTORS; i++) { + longView.put(i, i * 100L); // Use custom IDs + } + + index.addWithIds(NUM_VECTORS, vectorBuffer, idBuffer); + assertEquals(NUM_VECTORS, index.getCount()); + + index.build(BUILD_LIST_SIZE); + + // Search should return our custom IDs + float[] queryVectors = createQueryVectors(1, DIMENSION); + float[] distances = new float[K]; + long[] labels = new long[K]; + + index.search(1, queryVectors, K, SEARCH_LIST_SIZE, distances, labels); + + for (int i = 0; i < K; i++) { + assertTrue(labels[i] % 100 == 0, "Label should be a multiple of 100"); + } + } + } + + @Test + void testBatchSearch() { + try (Index index = createIndex(MetricType.L2)) { + addVectorsWithIds(index, NUM_VECTORS, DIMENSION, 0); + index.build(BUILD_LIST_SIZE); + + int numQueries = 5; + float[] queryVectors = createQueryVectors(numQueries, DIMENSION); + float[] distances = new float[numQueries * K]; + long[] labels = new long[numQueries * K]; + + index.search(numQueries, queryVectors, K, SEARCH_LIST_SIZE, distances, labels); + + // Read results for each query + for (int q = 0; q < numQueries; q++) { + for (int n = 0; n < K; n++) { + int idx = q * K + n; + assertTrue(labels[idx] >= 0 && labels[idx] < NUM_VECTORS); + assertTrue(distances[idx] >= 0); + } + } + } + } + + @Test + void testInnerProductMetric() { + try (Index index = createIndex(MetricType.INNER_PRODUCT)) { + assertEquals(MetricType.INNER_PRODUCT, index.getMetricType()); + + addVectorsWithIds(index, NUM_VECTORS, DIMENSION, 0); + index.build(BUILD_LIST_SIZE); + + float[] queryVectors = createQueryVectors(1, DIMENSION); + float[] distances = new float[K]; + long[] labels = new long[K]; + + index.search(1, queryVectors, K, SEARCH_LIST_SIZE, distances, labels); + + // DiskANN uses distance form for all metrics (lower = closer/more similar). + // For inner product the distance is derived so that results are still in + // ascending order by distance (the most similar result first). + for (int i = 1; i < K; i++) { + assertTrue( + distances[i] >= distances[i - 1], + "Distances should be sorted in ascending order (lower = more similar)"); + } + } + } + + @Test + void testCosineMetric() { + try (Index index = createIndex(MetricType.COSINE)) { + assertEquals(MetricType.COSINE, index.getMetricType()); + + addVectorsWithIds(index, NUM_VECTORS, DIMENSION, 0); + index.build(BUILD_LIST_SIZE); + + float[] queryVectors = createQueryVectors(1, DIMENSION); + float[] distances = new float[K]; + long[] labels = new long[K]; + + index.search(1, queryVectors, K, SEARCH_LIST_SIZE, distances, labels); + + // Cosine distance should be in [0, 2] range + for (int i = 0; i < K; i++) { + assertTrue(labels[i] >= 0, "Label should be non-negative"); + } + } + } + + @Test + void testSerialization() { + float[] queryVectors = createQueryVectors(1, DIMENSION); + long[] originalLabels = new long[K]; + float[] originalDistances = new float[K]; + + // Create, populate, and search + try (Index index = createIndex(MetricType.L2)) { + addVectorsWithIds(index, NUM_VECTORS, DIMENSION, 0); + index.build(BUILD_LIST_SIZE); + + index.search(1, queryVectors, K, SEARCH_LIST_SIZE, originalDistances, originalLabels); + + // Serialize + long serializeSize = index.serializeSize(); + assertTrue(serializeSize > 0); + + ByteBuffer serialized = + ByteBuffer.allocateDirect((int) serializeSize).order(ByteOrder.nativeOrder()); + long bytesWritten = index.serialize(serialized); + assertEquals(serializeSize, bytesWritten); + + // Convert to byte array for deserialization + serialized.rewind(); + byte[] serializedBytes = new byte[(int) bytesWritten]; + serialized.get(serializedBytes); + + // Deserialize and verify + try (Index deserializedIndex = Index.deserialize(serializedBytes)) { + assertEquals(DIMENSION, deserializedIndex.getDimension()); + assertEquals(NUM_VECTORS, deserializedIndex.getCount()); + + float[] deserializedDistances = new float[K]; + long[] deserializedLabels = new long[K]; + deserializedIndex.search( + 1, + queryVectors, + K, + SEARCH_LIST_SIZE, + deserializedDistances, + deserializedLabels); + + assertArrayEquals(originalLabels, deserializedLabels); + } + } + } + + @Test + void testSmallIndex() { + int dim = 2; + try (Index index = + Index.create(dim, MetricType.L2, INDEX_TYPE_MEMORY, MAX_DEGREE, BUILD_LIST_SIZE)) { + // Add a few vectors + ByteBuffer vectorBuffer = Index.allocateVectorBuffer(3, dim); + FloatBuffer floatView = vectorBuffer.asFloatBuffer(); + floatView.put(0, 1.0f); + floatView.put(1, 0.0f); // [1, 0] + floatView.put(2, 0.0f); + floatView.put(3, 1.0f); // [0, 1] + floatView.put(4, 0.7f); + floatView.put(5, 0.7f); // [0.7, 0.7] + + ByteBuffer idBuffer = Index.allocateIdBuffer(3); + LongBuffer longView = idBuffer.asLongBuffer(); + longView.put(0, 10L); + longView.put(1, 20L); + longView.put(2, 30L); + + index.addWithIds(3, vectorBuffer, idBuffer); + index.build(BUILD_LIST_SIZE); + + // Query for [1, 0] - should find ID 10 as nearest + float[] query = {1.0f, 0.0f}; + float[] distances = new float[1]; + long[] labels = new long[1]; + index.search(1, query, 1, SEARCH_LIST_SIZE, distances, labels); + + assertEquals(10L, labels[0], "Nearest to [1,0] should be ID 10"); + assertEquals(0.0f, distances[0], 1e-5f, "Distance to self should be ~0"); + } + } + + @Test + void testSearchResultArrays() { + try (Index index = createIndex(MetricType.L2)) { + addVectorsWithIds(index, 100, DIMENSION, 0); + index.build(BUILD_LIST_SIZE); + + int numQueries = 3; + int k = 5; + float[] queryVectors = createQueryVectors(numQueries, DIMENSION); + float[] distances = new float[numQueries * k]; + long[] labels = new long[numQueries * k]; + + index.search(numQueries, queryVectors, k, SEARCH_LIST_SIZE, distances, labels); + + // Test reading individual results + for (int q = 0; q < numQueries; q++) { + for (int n = 0; n < k; n++) { + int idx = q * k + n; + assertTrue(labels[idx] >= 0 && labels[idx] < 100); + assertTrue(distances[idx] >= 0); + } + } + } + } + + @Test + void testBufferAllocationHelpers() { + // Test vector buffer allocation + ByteBuffer vectorBuffer = Index.allocateVectorBuffer(10, DIMENSION); + assertTrue(vectorBuffer.isDirect()); + assertEquals(ByteOrder.nativeOrder(), vectorBuffer.order()); + assertEquals(10 * DIMENSION * Float.BYTES, vectorBuffer.capacity()); + + // Test ID buffer allocation + ByteBuffer idBuffer = Index.allocateIdBuffer(10); + assertTrue(idBuffer.isDirect()); + assertEquals(ByteOrder.nativeOrder(), idBuffer.order()); + assertEquals(10 * Long.BYTES, idBuffer.capacity()); + } + + @Test + void testErrorHandling() { + // Test buffer validation - wrong size buffer + try (Index index = createIndex(MetricType.L2)) { + ByteBuffer wrongSizeBuffer = + ByteBuffer.allocateDirect(10).order(ByteOrder.nativeOrder()); + ByteBuffer idBuffer = Index.allocateIdBuffer(1); + assertThrows( + IllegalArgumentException.class, + () -> { + index.addWithIds(1, wrongSizeBuffer, idBuffer); + }); + } + + // Test non-direct buffer + try (Index index = createIndex(MetricType.L2)) { + ByteBuffer heapBuffer = ByteBuffer.allocate(DIMENSION * Float.BYTES); + ByteBuffer idBuffer = Index.allocateIdBuffer(1); + assertThrows( + IllegalArgumentException.class, + () -> { + index.addWithIds(1, heapBuffer, idBuffer); + }); + } + + // Test serialize with non-direct buffer + try (Index index = createIndex(MetricType.L2)) { + ByteBuffer heapBuffer = ByteBuffer.allocate(100); + assertThrows( + IllegalArgumentException.class, + () -> { + index.serialize(heapBuffer); + }); + } + + // Test closed index + Index closedIndex = createIndex(MetricType.L2); + closedIndex.close(); + assertThrows( + IllegalStateException.class, + () -> { + closedIndex.getCount(); + }); + } + + @Test + void testQueryVectorArrayValidation() { + try (Index index = createIndex(MetricType.L2)) { + addVectorsWithIds(index, 10, DIMENSION, 0); + index.build(BUILD_LIST_SIZE); + + // Query vectors array too small + float[] tooSmall = new float[DIMENSION - 1]; + float[] distances = new float[K]; + long[] labels = new long[K]; + assertThrows( + IllegalArgumentException.class, + () -> { + index.search(1, tooSmall, K, SEARCH_LIST_SIZE, distances, labels); + }); + + // Distances array too small + float[] query = createQueryVectors(1, DIMENSION); + float[] smallDistances = new float[K - 1]; + assertThrows( + IllegalArgumentException.class, + () -> { + index.search(1, query, K, SEARCH_LIST_SIZE, smallDistances, labels); + }); + + // Labels array too small + long[] smallLabels = new long[K - 1]; + assertThrows( + IllegalArgumentException.class, + () -> { + index.search(1, query, K, SEARCH_LIST_SIZE, distances, smallLabels); + }); + } + } + + private Index createIndex(MetricType metricType) { + return Index.create(DIMENSION, metricType, INDEX_TYPE_MEMORY, MAX_DEGREE, BUILD_LIST_SIZE); + } + + /** Add vectors with sequential IDs starting from {@code startId}. */ + private void addVectorsWithIds(Index index, int n, int d, long startId) { + ByteBuffer vectorBuffer = createVectorBuffer(n, d); + ByteBuffer idBuffer = Index.allocateIdBuffer(n); + LongBuffer longView = idBuffer.asLongBuffer(); + for (int i = 0; i < n; i++) { + longView.put(i, startId + i); + } + index.addWithIds(n, vectorBuffer, idBuffer); + } + + /** Create a direct ByteBuffer with random vectors. */ + private ByteBuffer createVectorBuffer(int n, int d) { + ByteBuffer buffer = Index.allocateVectorBuffer(n, d); + FloatBuffer floatView = buffer.asFloatBuffer(); + + Random random = new Random(42); + for (int i = 0; i < n * d; i++) { + floatView.put(i, random.nextFloat()); + } + + return buffer; + } + + /** Create a float array with random query vectors. */ + private float[] createQueryVectors(int n, int d) { + float[] vectors = new float[n * d]; + Random random = new Random(42); + for (int i = 0; i < n * d; i++) { + vectors[i] = random.nextFloat(); + } + return vectors; + } +} diff --git a/paimon-diskann/pom.xml b/paimon-diskann/pom.xml new file mode 100644 index 000000000000..55f5d45df16f --- /dev/null +++ b/paimon-diskann/pom.xml @@ -0,0 +1,40 @@ + + + + 4.0.0 + + + paimon-parent + org.apache.paimon + 1.4-SNAPSHOT + + + paimon-diskann + Paimon : DiskANN + pom + + + paimon-diskann-jni + paimon-diskann-index + paimon-diskann-e2e-test + + diff --git a/pom.xml b/pom.xml index 52bbfac148ba..6fee18b0c081 100644 --- a/pom.xml +++ b/pom.xml @@ -532,6 +532,15 @@ under the License. true + + paimon-diskann + + paimon-diskann + + + true + + From 9f340644b0ebd0dc843b66f42da87473a70e819e Mon Sep 17 00:00:00 2001 From: yantian Date: Tue, 10 Feb 2026 17:10:11 +0800 Subject: [PATCH 2/7] add parameter config --- .../actions/build-diskann-native/action.yml | 6 + .github/workflows/build-diskann-native.yml | 2 +- .github/workflows/utitcase.yml | 29 +- paimon-diskann/PARAMETER_TUNING.md | 195 +++++++++ .../paimon/diskann/index/DiskAnnIndex.java | 46 +- .../index/DiskAnnVectorGlobalIndexReader.java | 52 ++- .../index/DiskAnnVectorGlobalIndexWriter.java | 23 +- paimon-diskann/paimon-diskann-jni/pom.xml | 4 + .../scripts/build-native.sh | 230 +++++++++- .../java/org/apache/paimon/diskann/Index.java | 2 +- .../paimon/diskann/NativeLibraryLoader.java | 18 +- .../src/main/native/Cargo.toml | 4 + .../src/main/native/src/lib.rs | 395 ++++++++---------- .../aarch64/libpaimon_diskann_jni.dylib | Bin 1023296 -> 1024496 bytes 14 files changed, 763 insertions(+), 243 deletions(-) create mode 100644 paimon-diskann/PARAMETER_TUNING.md diff --git a/.github/actions/build-diskann-native/action.yml b/.github/actions/build-diskann-native/action.yml index c350844fbf08..347cfb82c769 100644 --- a/.github/actions/build-diskann-native/action.yml +++ b/.github/actions/build-diskann-native/action.yml @@ -21,6 +21,12 @@ runs: rustc --version cargo --version + - name: Install dependency bundling tools (Linux) + if: startsWith(inputs.platform, 'linux') + shell: bash + run: | + sudo apt-get update -qq && sudo apt-get install -y -qq patchelf + - name: Build native library shell: bash run: | diff --git a/.github/workflows/build-diskann-native.yml b/.github/workflows/build-diskann-native.yml index 0495fe50b7e6..f4e7ae00b540 100644 --- a/.github/workflows/build-diskann-native.yml +++ b/.github/workflows/build-diskann-native.yml @@ -64,5 +64,5 @@ jobs: uses: actions/upload-artifact@v6 with: name: ${{ inputs.artifact-name }} - path: paimon-diskann/paimon-diskann-jni/src/main/resources/linux/amd64/ + path: paimon-diskann/paimon-diskann-jni/src/main/resources/ retention-days: ${{ inputs.retention-days }} diff --git a/.github/workflows/utitcase.yml b/.github/workflows/utitcase.yml index 915ec0385ac9..b64c169cf7b6 100644 --- a/.github/workflows/utitcase.yml +++ b/.github/workflows/utitcase.yml @@ -29,6 +29,8 @@ on: - 'paimon-lucene/**' - 'paimon-faiss/**' - '.github/workflows/faiss-vector-index-tests.yml' + - 'paimon-diskann/**' + - '.github/workflows/publish-diskann_snapshot.yml' - 'paimon-core/src/test/java/org/apache/paimon/JavaPyE2ETest.java' env: @@ -40,15 +42,21 @@ concurrency: cancel-in-progress: true jobs: - build_native: + build_faiss_native: uses: ./.github/workflows/build-faiss-native.yml with: platform: linux-amd64 jdk-version: '8' + build_diskann_native: + uses: ./.github/workflows/build-diskann-native.yml + with: + platform: linux-amd64 + jdk-version: '8' + build_test: runs-on: ubuntu-latest - needs: build_native + needs: [build_faiss_native, build_diskann_native] steps: - name: Checkout code @@ -60,21 +68,30 @@ jobs: java-version: ${{ env.JDK_VERSION }} distribution: 'temurin' - - name: Download native library artifact + - name: Download FAISS native library artifact uses: actions/download-artifact@v7 with: name: faiss-native-linux-amd64 path: paimon-faiss/paimon-faiss-jni/src/main/resources/linux/amd64/ - - name: List downloaded native library + - name: Download DiskANN native library artifact + uses: actions/download-artifact@v7 + with: + name: diskann-native-linux-amd64 + path: paimon-diskann/paimon-diskann-jni/src/main/resources/ + + - name: List downloaded native libraries run: | - echo "=== Downloaded native libraries ===" + echo "=== FAISS native libraries ===" ls -la paimon-faiss/paimon-faiss-jni/src/main/resources/linux/amd64/ + echo "" + echo "=== DiskANN native libraries ===" + find paimon-diskann/paimon-diskann-jni/src/main/resources -type f -exec ls -la {} \; - name: Build Others run: | echo "Start compiling modules" - mvn -T 2C -B -ntp clean install -DskipTests -Pflink1,spark3,paimon-faiss + mvn -T 2C -B -ntp clean install -DskipTests -Pflink1,spark3,paimon-faiss,paimon-diskann - name: Test Others timeout-minutes: 60 diff --git a/paimon-diskann/PARAMETER_TUNING.md b/paimon-diskann/PARAMETER_TUNING.md new file mode 100644 index 000000000000..61e0a5126112 --- /dev/null +++ b/paimon-diskann/PARAMETER_TUNING.md @@ -0,0 +1,195 @@ + + +# DiskANN Parameter Tuning Guide + +This document provides guidance on tuning DiskANN vector index parameters for optimal performance in Apache Paimon. + +## Overview + +DiskANN is a graph-based approximate nearest neighbor (ANN) search algorithm designed for efficient billion-point vector search. The implementation in Paimon provides several parameters to control the trade-offs between accuracy, speed, and resource usage. + +## Key Parameters + +### 1. Graph Construction Parameters + +#### `vector.diskann.max-degree` (R) +- **Default**: 64 +- **Range**: 32-128 +- **Description**: Maximum degree (number of connections) for each node in the graph +- **Impact**: + - Higher values → Better recall, higher memory usage, longer build time + - Lower values → Faster build, lower memory, potentially lower recall +- **Recommendations**: + - **32**: For memory-constrained environments or when build time is critical + - **64**: Balanced default (Microsoft recommended) + - **128**: For maximum recall when resources permit + +#### `vector.diskann.build-list-size` (L) +- **Default**: 100 +- **Range**: 50-200 +- **Description**: Size of the candidate list during graph construction +- **Impact**: + - Higher values → Better graph quality, longer build time + - Lower values → Faster build, potentially lower recall +- **Recommendations**: + - Use default 100 for most cases + - Increase to 150-200 for very high-dimensional data (>512 dimensions) + +### 2. Search Parameters + +#### `vector.diskann.search-list-size` (L) +- **Default**: 100 +- **Range**: 16-500 +- **Description**: Size of the candidate list during search +- **Impact**: + - Higher values → Better recall, higher latency + - Lower values → Lower latency, potentially lower recall +- **Dynamic Behavior**: The implementation automatically adjusts this to be at least equal to the requested `k` (number of results) +- **Recommendations**: + - **16-32**: For latency-critical applications (QPS > 5000) + - **100**: Balanced default + - **200-500**: For maximum recall (recall > 95%) + +#### `vector.search-factor` +- **Default**: 10 +- **Range**: 5-20 +- **Description**: Multiplier for search limit when row filtering is applied +- **Impact**: When filtering by row IDs, fetches `limit * search-factor` results to ensure sufficient matches after filtering +- **Recommendations**: + - **5**: When filtering is selective (<10% of data) + - **10**: Default for typical filtering scenarios + - **20**: When filtering is very broad (>50% of data) + +### 3. Data Configuration + +#### `vector.dim` +- **Default**: 128 +- **Description**: Dimension of the vectors +- **Recommendations**: + - Must match your embedding model + - Common values: 128, 256, 384, 512, 768, 1024 + +#### `vector.metric` +- **Default**: L2 +- **Options**: L2, INNER_PRODUCT, COSINE +- **Description**: Distance metric for similarity computation +- **Recommendations**: + - **L2**: For Euclidean distance (most common) + - **INNER_PRODUCT**: For dot product similarity (use with normalized vectors) + - **COSINE**: For cosine similarity + +#### `vector.normalize` +- **Default**: false +- **Description**: Whether to L2-normalize vectors before indexing/searching +- **Recommendations**: + - **true**: When using COSINE metric or when vectors have varying magnitudes + - **false**: When vectors are already normalized or using L2 metric + +### 4. Index Organization + +#### `vector.size-per-index` +- **Default**: 2,000,000 +- **Description**: Number of vectors per index file +- **Impact**: + - Larger values → Fewer files, higher memory per index, better search efficiency + - Smaller values → More files, lower memory per index, more overhead +- **Recommendations**: + - **500,000**: For small datasets or memory-constrained environments + - **2,000,000**: Default for balanced performance + - **5,000,000+**: For large-scale production systems with ample resources + +#### `vector.diskann.index-type` +- **Default**: MEMORY +- **Options**: MEMORY, DISK +- **Description**: Type of index structure +- **Recommendations**: + - **MEMORY**: For datasets that fit in RAM (best performance) + - **DISK**: For datasets exceeding RAM (requires SSD) + +## Performance Tuning Guide + +### High Recall (>95%) +```properties +vector.diskann.max-degree = 128 +vector.diskann.build-list-size = 150 +vector.diskann.search-list-size = 200 +``` + +### Balanced (90-95% recall) +```properties +vector.diskann.max-degree = 64 +vector.diskann.build-list-size = 100 +vector.diskann.search-list-size = 100 +``` + +### High QPS (Low Latency) +```properties +vector.diskann.max-degree = 32 +vector.diskann.build-list-size = 75 +vector.diskann.search-list-size = 32 +``` + +### Memory-Constrained +```properties +vector.diskann.max-degree = 32 +vector.diskann.build-list-size = 75 +vector.size-per-index = 500000 +vector.diskann.index-type = DISK +``` + +## Best Practices + +1. **Start with defaults**: The default parameters are tuned for balanced performance +2. **Measure first**: Profile your workload before tuning +3. **Tune incrementally**: Change one parameter at a time and measure impact +4. **Consider trade-offs**: Higher recall typically means higher latency and resource usage +5. **Test with production data**: Parameter effectiveness depends on data characteristics + +## Advanced Parameters (Future Enhancement) + +The following parameters are documented in the official Microsoft DiskANN implementation but are not yet exposed in the current Rust-based native library: + +- **alpha** (default: 1.2): Controls the graph construction pruning strategy +- **saturate_graph** (default: true): Whether to saturate the graph during construction + +These parameters may be added in future versions when the underlying Rust DiskANN crate exposes them through its configuration API. + +## Performance Metrics + +When tuning parameters, monitor these metrics: +- **Recall**: Percentage of true nearest neighbors found +- **QPS (Queries Per Second)**: Throughput of search operations +- **Latency**: Time to complete a single query (p50, p95, p99) +- **Memory Usage**: RAM consumed by indices +- **Build Time**: Time to construct the index + +## Recent Improvements + +### Dynamic Search List Sizing (v1.0+) +The search list size is now automatically adjusted to be at least equal to the requested `k`. This follows Milvus best practices and ensures optimal recall without manual tuning. + +### Memory-Efficient Loading (v1.0+) +Indices are now loaded through temporary files, allowing the OS to manage memory more efficiently for large indices. This is a step toward full mmap support. + +## References + +- [Microsoft DiskANN Paper](https://proceedings.neurips.cc/paper/2019/file/09853c7fb1d3f8ee67a61b6bf4a7f8e6-Paper.pdf) +- [Microsoft DiskANN Library](https://github.com/microsoft/DiskANN) +- [Milvus DiskANN Documentation](https://milvus.io/docs/diskann.md) diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java index a83c23ed5e3d..ae0fd9a536f1 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java @@ -37,14 +37,23 @@ public class DiskAnnIndex implements Closeable { private final int dimension; private final DiskAnnVectorMetric metric; private final DiskAnnIndexType indexType; + private final int maxDegree; + private final int buildListSize; private volatile boolean closed = false; private DiskAnnIndex( - Index index, int dimension, DiskAnnVectorMetric metric, DiskAnnIndexType indexType) { + Index index, + int dimension, + DiskAnnVectorMetric metric, + DiskAnnIndexType indexType, + int maxDegree, + int buildListSize) { this.index = index; this.dimension = dimension; this.metric = metric; this.indexType = indexType; + this.maxDegree = maxDegree; + this.buildListSize = buildListSize; } public static DiskAnnIndex create( @@ -56,7 +65,7 @@ public static DiskAnnIndex create( MetricType metricType = metric.toMetricType(); Index index = Index.create(dimension, metricType, indexType.value(), maxDegree, buildListSize); - return new DiskAnnIndex(index, dimension, metric, indexType); + return new DiskAnnIndex(index, dimension, metric, indexType, maxDegree, buildListSize); } public void addWithIds(ByteBuffer vectorBuffer, ByteBuffer idBuffer, int n) { @@ -66,7 +75,12 @@ public void addWithIds(ByteBuffer vectorBuffer, ByteBuffer idBuffer, int n) { index.addWithIds(n, vectorBuffer, idBuffer); } - public void build(int buildListSize) { + /** + * Build the index graph after adding vectors. + * + *

Uses the buildListSize parameter that was specified during index creation. + */ + public void build() { ensureOpen(); index.build(buildListSize); } @@ -114,6 +128,14 @@ public DiskAnnIndexType indexType() { return indexType; } + public int maxDegree() { + return maxDegree; + } + + public int buildListSize() { + return buildListSize; + } + public long serializeSize() { ensureOpen(); return index.serializeSize(); @@ -129,7 +151,23 @@ public long serialize(ByteBuffer buffer) { public static DiskAnnIndex deserialize(byte[] data, DiskAnnVectorMetric metric) { Index index = Index.deserialize(data); - return new DiskAnnIndex(index, index.getDimension(), metric, DiskAnnIndexType.UNKNOWN); + return new DiskAnnIndex( + index, index.getDimension(), metric, DiskAnnIndexType.UNKNOWN, 64, 100); + } + + /** + * Reset the index (remove all vectors). + * + *

Note: This is not supported in the current implementation. DiskANN indices are immutable + * once built. To "reset", you must create a new index. + * + * @throws UnsupportedOperationException always, as reset is not currently supported + */ + public void reset() { + throw new UnsupportedOperationException( + "Reset is not supported for DiskANN indices. " + + "DiskANN indices are immutable once built. " + + "Please create a new index instead."); } public static ByteBuffer allocateVectorBuffer(int numVectors, int dimension) { diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java index 1c505fa91509..dac7db526016 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java @@ -31,23 +31,30 @@ import org.apache.paimon.utils.IOUtils; import org.apache.paimon.utils.RoaringNavigableMap64; +import java.io.File; +import java.io.FileOutputStream; import java.io.IOException; +import java.nio.file.Files; import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Optional; import java.util.PriorityQueue; +import java.util.UUID; /** * Vector global index reader using DiskANN. * - *

This implementation uses DiskANN for efficient approximate nearest neighbor search. + *

This implementation uses DiskANN for efficient approximate nearest neighbor search. It + * supports lazy loading of indices and optional memory-mapped file loading for better memory + * efficiency with large indices. */ public class DiskAnnVectorGlobalIndexReader implements GlobalIndexReader { private final List indices; private final List indexMetas; + private final List localIndexFiles; private final List ioMetas; private final GlobalIndexFileReader fileReader; private final DataType fieldType; @@ -66,6 +73,7 @@ public DiskAnnVectorGlobalIndexReader( this.options = options; this.indices = new ArrayList<>(); this.indexMetas = new ArrayList<>(); + this.localIndexFiles = new ArrayList<>(); } @Override @@ -144,7 +152,10 @@ private GlobalIndexResult search(VectorSearch vectorSearch) throws IOException { float[] distances = new float[effectiveK]; long[] labels = new long[effectiveK]; - index.search(queryVector, 1, effectiveK, options.searchListSize(), distances, labels); + // Dynamic search list sizing: use max of configured value and effectiveK + // This follows Milvus best practice: search_list should be >= topk + int dynamicSearchListSize = Math.max(options.searchListSize(), effectiveK); + index.search(queryVector, 1, effectiveK, dynamicSearchListSize, distances, labels); for (int i = 0; i < effectiveK; i++) { long rowId = labels[i]; @@ -259,7 +270,25 @@ private void loadIndexAt(int position) throws IOException { } private DiskAnnIndex loadIndex(SeekableInputStream in) throws IOException { - byte[] data = IOUtils.readFully(in, true); + // For better memory efficiency, write to a temporary file + // This allows the OS to manage memory more efficiently for large indices + File tempIndexFile = + Files.createTempFile("paimon-diskann-" + UUID.randomUUID(), ".index").toFile(); + localIndexFiles.add(tempIndexFile); + + // Copy index data to temp file + try (FileOutputStream fos = new FileOutputStream(tempIndexFile)) { + byte[] buffer = new byte[32768]; + int bytesRead; + while ((bytesRead = in.read(buffer)) != -1) { + fos.write(buffer, 0, bytesRead); + } + } + + // Load from file for potential mmap benefits + // Note: Current implementation still deserializes to memory + // Future enhancement: Add native file-based loading if supported + byte[] data = Files.readAllBytes(tempIndexFile.toPath()); return DiskAnnIndex.deserialize(data, options.metric()); } @@ -280,6 +309,7 @@ private void normalizeL2(float[] vector) { public void close() throws IOException { Throwable firstException = null; + // Close all DiskANN indices for (DiskAnnIndex index : indices) { if (index == null) { continue; @@ -296,6 +326,22 @@ public void close() throws IOException { } indices.clear(); + // Delete temporary files + for (File tempFile : localIndexFiles) { + try { + if (tempFile != null && tempFile.exists()) { + tempFile.delete(); + } + } catch (Throwable t) { + if (firstException == null) { + firstException = t; + } else { + firstException.addSuppressed(t); + } + } + } + localIndexFiles.clear(); + if (firstException != null) { if (firstException instanceof IOException) { throw (IOException) firstException; diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java index 16d709345d32..ca60a59babc9 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java @@ -27,6 +27,7 @@ import org.apache.paimon.types.FloatType; import java.io.BufferedOutputStream; +import java.io.Closeable; import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -41,8 +42,11 @@ * *

Vectors are added to the index in batches. When the current index reaches {@code sizePerIndex} * vectors, it is built and serialized to a file and a new index is created. + * + *

This class implements {@link Closeable} so that the native DiskANN index is released even if + * {@link #finish()} is never called or throws an exception. */ -public class DiskAnnVectorGlobalIndexWriter implements GlobalIndexSingletonWriter { +public class DiskAnnVectorGlobalIndexWriter implements GlobalIndexSingletonWriter, Closeable { private static final int DEFAULT_BATCH_SIZE = 10000; @@ -174,7 +178,7 @@ private void flushCurrentIndex() throws IOException { } if (!built) { - currentIndex.build(options.buildListSize()); + currentIndex.build(); built = true; } @@ -247,6 +251,21 @@ private void normalizeL2(float[] vector) { } } + /** + * Release native resources held by the current in-progress index. + * + *

This is a safety net: under normal operation the index is closed by {@link + * #flushCurrentIndex()}, but if an error occurs before flushing this method ensures the native + * handle is freed. + */ + @Override + public void close() { + if (currentIndex != null) { + currentIndex.close(); + currentIndex = null; + } + } + /** Entry holding a vector and its row ID. */ private static class VectorEntry { final long id; diff --git a/paimon-diskann/paimon-diskann-jni/pom.xml b/paimon-diskann/paimon-diskann-jni/pom.xml index b655689e756a..630f31e3b67e 100644 --- a/paimon-diskann/paimon-diskann-jni/pom.xml +++ b/paimon-diskann/paimon-diskann-jni/pom.xml @@ -54,6 +54,8 @@ under the License. so + + so.* dylib @@ -85,6 +87,8 @@ under the License. ${project.basedir}/src/main/resources **/*.so + + **/*.so.* **/*.dylib diff --git a/paimon-diskann/paimon-diskann-jni/scripts/build-native.sh b/paimon-diskann/paimon-diskann-jni/scripts/build-native.sh index 026daa7cfc45..61763a8129a7 100755 --- a/paimon-diskann/paimon-diskann-jni/scripts/build-native.sh +++ b/paimon-diskann/paimon-diskann-jni/scripts/build-native.sh @@ -193,9 +193,235 @@ fi cp "$SRC_LIB" "$OUTPUT_DIR/" +# ===================================================================== +# Bundle shared library dependencies +# ===================================================================== +# Rust cdylib statically links all Rust code but may dynamically link +# to system C/C++ libraries (libgcc_s, libstdc++, etc.). On Linux CI +# containers the target machine may have different versions, so we +# bundle all non-trivial dependencies — mirroring the FAISS approach. +# ===================================================================== + echo "" -echo "Native library location:" -ls -la "$OUTPUT_DIR/$(basename "$SRC_LIB")" +echo "============================================" +echo "Checking & bundling library dependencies" +echo "============================================" + +if [ "$OS" = "Linux" ]; then + # ---- Helper: copy a real library file into OUTPUT_DIR ---- + bundle_lib() { + local src_path="$1" + local target_name="$2" + + if [ -f "$OUTPUT_DIR/$target_name" ]; then + echo " Already bundled: $target_name" + return 0 + fi + + # Resolve symlinks to the real file + local real_path + real_path=$(readlink -f "$src_path" 2>/dev/null || realpath "$src_path" 2>/dev/null || echo "$src_path") + if [ ! -f "$real_path" ]; then + echo " Cannot resolve: $src_path" + return 1 + fi + + cp "$real_path" "$OUTPUT_DIR/$target_name" + chmod +x "$OUTPUT_DIR/$target_name" + echo " Bundled: $real_path -> $target_name" + return 0 + } + + # ---- Helper: search common paths for a library by glob pattern ---- + find_and_bundle() { + local pattern="$1" + local target_name="$2" + + if [ -f "$OUTPUT_DIR/$target_name" ]; then + echo " Already bundled: $target_name" + return 0 + fi + + for search_path in /usr/local/lib /usr/local/lib64 \ + /usr/lib /usr/lib64 \ + /usr/lib/x86_64-linux-gnu /usr/lib/aarch64-linux-gnu; do + local found_lib + found_lib=$(find "$search_path" -maxdepth 1 -name "$pattern" -type f 2>/dev/null | head -1) + if [ -n "$found_lib" ] && [ -f "$found_lib" ]; then + bundle_lib "$found_lib" "$target_name" + return $? + fi + local found_link + found_link=$(find "$search_path" -maxdepth 1 -name "$pattern" -type l 2>/dev/null | head -1) + if [ -n "$found_link" ] && [ -L "$found_link" ]; then + bundle_lib "$found_link" "$target_name" + return $? + fi + done + + # Try ldconfig cache + local ldconfig_path + ldconfig_path=$(ldconfig -p 2>/dev/null | grep "$pattern" | head -1 | awk '{print $NF}') + if [ -n "$ldconfig_path" ] && [ -f "$ldconfig_path" ]; then + bundle_lib "$ldconfig_path" "$target_name" + return $? + fi + + echo " Not found: $pattern" + return 1 + } + + echo "" + echo "Bundling required libraries..." + + # 1. GCC runtime (Rust cdylib may link against libgcc_s for stack unwinding) + if ! find_and_bundle "libgcc_s.so*" "libgcc_s.so.1"; then + echo " Note: libgcc_s not found as shared library - likely statically linked" + fi + + # 2. C++ standard library (needed if the diskann crate compiles any C++ code) + if ! find_and_bundle "libstdc++.so*" "libstdc++.so.6"; then + echo " Note: libstdc++ not found as shared library - likely statically linked" + fi + + # ---- Scan ldd for additional non-system dependencies ---- + echo "" + echo "Scanning ldd for additional dependencies..." + JNI_LIB="$OUTPUT_DIR/$(basename "$SRC_LIB")" + LIBS_TO_CHECK="$JNI_LIB" + for bundled_lib in "$OUTPUT_DIR"/*.so*; do + [ -f "$bundled_lib" ] && LIBS_TO_CHECK="$LIBS_TO_CHECK $bundled_lib" + done + + LIBS_CHECKED="" + while [ -n "$LIBS_TO_CHECK" ]; do + CURRENT_LIB=$(echo "$LIBS_TO_CHECK" | awk '{print $1}') + LIBS_TO_CHECK=$(echo "$LIBS_TO_CHECK" | cut -d' ' -f2-) + [ "$LIBS_TO_CHECK" = "$CURRENT_LIB" ] && LIBS_TO_CHECK="" + + # Skip already-checked + echo "$LIBS_CHECKED" | grep -q "$CURRENT_LIB" 2>/dev/null && continue + LIBS_CHECKED="$LIBS_CHECKED $CURRENT_LIB" + + [ ! -f "$CURRENT_LIB" ] && continue + + echo " Checking deps of: $(basename "$CURRENT_LIB")" + + DEPS=$(ldd "$CURRENT_LIB" 2>/dev/null | grep "=>" | awk '{print $1 " " $3}') || true + + while IFS= read -r dep_line; do + [ -z "$dep_line" ] && continue + DEP_NAME=$(echo "$dep_line" | awk '{print $1}') + DEP_PATH=$(echo "$dep_line" | awk '{print $2}') + + # Skip universally-available system libraries + case "$DEP_NAME" in + linux-vdso.so*|libc.so*|libm.so*|libpthread.so*|libdl.so*|librt.so*|ld-linux*) + continue + ;; + esac + + # Bundle known problematic libraries + case "$DEP_NAME" in + libgcc_s*) + bundle_lib "$DEP_PATH" "libgcc_s.so.1" || true + ;; + libstdc++*) + if bundle_lib "$DEP_PATH" "libstdc++.so.6"; then + LIBS_TO_CHECK="$LIBS_TO_CHECK $OUTPUT_DIR/libstdc++.so.6" + fi + ;; + libgomp*) + if bundle_lib "$DEP_PATH" "libgomp.so.1"; then + LIBS_TO_CHECK="$LIBS_TO_CHECK $OUTPUT_DIR/libgomp.so.1" + fi + ;; + libquadmath*) + bundle_lib "$DEP_PATH" "libquadmath.so.0" || true + ;; + libgfortran*) + bundle_lib "$DEP_PATH" "libgfortran.so.3" || true + ;; + esac + done <<< "$DEPS" + done + + # ---- Set rpath to $ORIGIN so bundled libs are found at load time ---- + if command -v patchelf &>/dev/null; then + echo "" + echo "Setting rpath to \$ORIGIN for all libraries..." + for lib in "$OUTPUT_DIR"/*.so*; do + if [ -f "$lib" ]; then + patchelf --set-rpath '$ORIGIN' "$lib" 2>/dev/null || true + fi + done + echo "Done setting rpath" + else + echo "" + echo "WARNING: patchelf not found, cannot set rpath." + echo " Install with: sudo apt-get install patchelf" + echo " The Java loader will still pre-load deps from JAR, but setting" + echo " rpath provides an additional safety net." + fi + +elif [ "$OS" = "Darwin" ]; then + # On macOS, Rust cdylibs are normally self-contained. + # But check if any non-system dylibs are referenced. + echo "" + echo "Checking macOS dylib dependencies..." + DYLIB_PATH="$OUTPUT_DIR/$(basename "$SRC_LIB")" + otool -L "$DYLIB_PATH" 2>/dev/null | tail -n +2 | while read -r dep_entry; do + dep_path=$(echo "$dep_entry" | awk '{print $1}') + case "$dep_path" in + /usr/lib/*|/System/*|@rpath/*|@loader_path/*|@executable_path/*) + # System or relative — OK + ;; + *) + if [ -f "$dep_path" ]; then + dep_basename=$(basename "$dep_path") + if [ ! -f "$OUTPUT_DIR/$dep_basename" ]; then + echo " Bundling macOS dep: $dep_path -> $dep_basename" + cp "$dep_path" "$OUTPUT_DIR/$dep_basename" + chmod +x "$OUTPUT_DIR/$dep_basename" + # Rewrite the install name so the JNI lib finds the bundled copy + install_name_tool -change "$dep_path" "@loader_path/$dep_basename" "$DYLIB_PATH" 2>/dev/null || true + fi + fi + ;; + esac + done +fi + +# ===================================================================== +# Summary: list all libraries and their dependencies +# ===================================================================== + +echo "" +echo "============================================" +echo "Native library summary" +echo "============================================" + +BUILT_LIBS=$(find "$PROJECT_DIR/src/main/resources" -type f \( -name "*.so" -o -name "*.so.*" -o -name "*.dylib" \) 2>/dev/null) + +if [ -n "$BUILT_LIBS" ]; then + for lib in $BUILT_LIBS; do + echo "" + echo "Library: $lib" + ls -la "$lib" + + echo "" + echo "Dependencies:" + if [ "$OS" = "Darwin" ]; then + otool -L "$lib" 2>/dev/null | head -20 || true + elif [ "$OS" = "Linux" ]; then + ldd "$lib" 2>/dev/null | head -20 || readelf -d "$lib" 2>/dev/null | grep NEEDED | head -20 || true + fi + done +else + echo " (no libraries found)" + ls -la "$PROJECT_DIR/src/main/resources/"*/*/ 2>/dev/null || true +fi + echo "" echo "To package the JAR with native libraries, run:" echo " mvn package" diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java index 6d5ddc059b67..1a9195856b93 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java @@ -153,7 +153,7 @@ private void checkNotClosed() { } @Override - public void close() { + public synchronized void close() { if (!closed) { closed = true; if (nativeHandle != 0) { diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/NativeLibraryLoader.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/NativeLibraryLoader.java index 187ee244e482..6369e8913fa3 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/NativeLibraryLoader.java +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/NativeLibraryLoader.java @@ -49,8 +49,22 @@ public class NativeLibraryLoader { /** System property to specify a custom path to the native library. */ private static final String LIBRARY_PATH_PROPERTY = "paimon.diskann.lib.path"; - /** Dependency libraries that need to be loaded before the main JNI library. */ - private static final String[] DEPENDENCY_LIBRARIES = {}; + /** + * Dependency libraries that need to be loaded before the main JNI library. These are bundled in + * the JAR when the build script detects they are dynamically linked. + * + *

Order matters! Libraries must be loaded before the libraries that depend on them. The Rust + * {@code cdylib} statically links all Rust code but may dynamically link against the GCC + * runtime and C++ standard library on Linux. + */ + private static final String[] DEPENDENCY_LIBRARIES = { + // GCC runtime (Rust cdylib uses libgcc_s for stack unwinding on Linux) + "libgcc_s.so.1", + // C++ standard library (needed if diskann crate compiles C++ code internally) + "libstdc++.so.6", + // OpenMP runtime (possible transitive dependency) + "libgomp.so.1", + }; /** Whether the native library has been loaded. */ private static volatile boolean libraryLoaded = false; diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml index 352815c18818..acd48263f9b6 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml @@ -24,6 +24,10 @@ crate-type = ["cdylib"] [dependencies] jni = "0.21" +# DiskANN with testing feature: Provides test_provider for in-memory vector storage +# Note: Despite the name "testing", this is used in production for the in-memory index implementation diskann = { version = "0.45.0", features = ["testing"] } diskann-vector = "0.45.0" +# Tokio with rt feature: Required for async DiskANN operations +# Uses single-threaded runtime (new_current_thread) for efficient resource usage tokio = { version = "1", features = ["rt"] } diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs index fbea9973e85c..5c3df469a148 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs @@ -22,12 +22,20 @@ //! This module uses Microsoft's official `diskann` Rust crate (v0.45.0) //! from to provide graph-based //! approximate nearest neighbor search via JNI. +//! +//! # JNI Safety +//! +//! Every `extern "system"` entry point is wrapped with [`std::panic::catch_unwind`] +//! so that a Rust panic never unwinds across the FFI boundary, which would cause +//! undefined behaviour and likely crash the JVM. On panic the function throws a +//! `java.lang.RuntimeException` with the panic message and returns a safe default. use jni::objects::{JByteArray, JByteBuffer, JClass, JPrimitiveArray, ReleaseMode}; use jni::sys::{jfloat, jint, jlong}; use jni::JNIEnv; use std::collections::HashMap; +use std::panic::{self, AssertUnwindSafe}; use std::sync::{Arc, Mutex, OnceLock}; use diskann::graph::test::provider as test_provider; @@ -47,9 +55,32 @@ const MAGIC: i32 = 0x5044414E; const SERIALIZE_VERSION: i32 = 2; /// The u32 ID reserved for the DiskANN graph start/entry point. -/// This is not a user vector and is filtered from search results. const START_POINT_ID: u32 = 0; +// ======================== Panic‐safe JNI helper ======================== + +/// Run `body` inside [`catch_unwind`]. If it panics, throw a Java +/// `RuntimeException` with the panic message and return `default`. +fn jni_catch_unwind(env: &mut JNIEnv, default: R, body: F) -> R +where + F: FnOnce() -> R + panic::UnwindSafe, +{ + match panic::catch_unwind(body) { + Ok(v) => v, + Err(payload) => { + let msg = if let Some(s) = payload.downcast_ref::<&str>() { + s.to_string() + } else if let Some(s) = payload.downcast_ref::() { + s.clone() + } else { + "Unknown Rust panic in DiskANN JNI".to_string() + }; + let _ = env.throw_new("java/lang/RuntimeException", msg); + default + } + } +} + // ======================== Metric Mapping ======================== fn map_metric(metric_type: i32) -> Metric { @@ -62,29 +93,21 @@ fn map_metric(metric_type: i32) -> Metric { // ======================== Index State ======================== -/// Holds the DiskANN index, tokio runtime, and ID mappings. struct IndexState { - /// The real DiskANN graph index backed by an in-memory test provider. index: Arc>, - /// Execution context for the test provider. context: test_provider::Context, - /// Tokio runtime for running async DiskANN operations. runtime: tokio::runtime::Runtime, - // -- Metadata -- dimension: i32, metric_type: i32, index_type: i32, max_degree: usize, build_list_size: usize, - // -- ID mapping (user i64 ↔ DiskANN u32) -- ext_to_int: HashMap, int_to_ext: HashMap, - /// Next u32 ID to assign (0 is reserved for start point). next_id: u32, - // -- Raw data kept for serialization -- raw_data: Vec<(i64, Vec)>, } @@ -116,7 +139,6 @@ fn registry() -> &'static Mutex { REGISTRY.get_or_init(|| Mutex::new(IndexRegistry::new())) } -/// Get a cloned Arc to the index state (brief registry lock). fn get_index(handle: i64) -> Option>> { let guard = registry().lock().ok()?; guard.indices.get(&handle).cloned() @@ -124,7 +146,6 @@ fn get_index(handle: i64) -> Option>> { // ======================== Index Construction ======================== -/// Create a new DiskANN index backed by the official `diskann` crate. fn create_index_state( dimension: i32, metric_type: i32, @@ -135,17 +156,13 @@ fn create_index_state( let dim = dimension as usize; let metric = map_metric(metric_type); let md = std::cmp::max(max_degree as usize, 4); - // l_build must be >= pruned_degree for the DiskANN config to validate. let bls = std::cmp::max(build_list_size as usize, md); - // Tokio runtime for async DiskANN operations. let runtime = tokio::runtime::Builder::new_current_thread() .enable_all() .build() .map_err(|e| format!("Failed to create tokio runtime: {}", e))?; - // The DiskANN graph needs at least one start/entry point. - // Use a non-zero vector to avoid division-by-zero with cosine metric. let start_vector = vec![1.0f32; dim]; let provider_config = test_provider::Config::new( metric, @@ -155,14 +172,11 @@ fn create_index_state( .map_err(|e| format!("Failed to create provider config: {:?}", e))?; let provider = test_provider::Provider::new(provider_config); - // Build graph config using the metric's default prune kind. - // Use MaxDegree::same() because the test provider enforces a strict max_degree - // and does not allow slack (the graph construction would exceed the provider limit). let index_config = graph::config::Builder::new( md, graph::config::MaxDegree::same(), bls, - metric.into(), // Metric → PruneKind conversion + metric.into(), ) .build() .map_err(|e| format!("Failed to create index config: {:?}", e))?; @@ -198,71 +212,56 @@ fn get_direct_buffer_slice<'a>( if capacity < len { return None; } - // SAFETY: The caller guarantees the buffer is valid for `len` bytes. unsafe { Some(std::slice::from_raw_parts_mut(ptr, len)) } } // ======================== Serialization Helpers ======================== fn read_i32(buf: &[u8], offset: &mut usize) -> Option { - if *offset + 4 > buf.len() { - return None; - } - let mut bytes = [0u8; 4]; - bytes.copy_from_slice(&buf[*offset..*offset + 4]); + if *offset + 4 > buf.len() { return None; } + let mut b = [0u8; 4]; + b.copy_from_slice(&buf[*offset..*offset + 4]); *offset += 4; - Some(i32::from_ne_bytes(bytes)) + Some(i32::from_ne_bytes(b)) } fn read_i64(buf: &[u8], offset: &mut usize) -> Option { - if *offset + 8 > buf.len() { - return None; - } - let mut bytes = [0u8; 8]; - bytes.copy_from_slice(&buf[*offset..*offset + 8]); + if *offset + 8 > buf.len() { return None; } + let mut b = [0u8; 8]; + b.copy_from_slice(&buf[*offset..*offset + 8]); *offset += 8; - Some(i64::from_ne_bytes(bytes)) + Some(i64::from_ne_bytes(b)) } fn read_f32(buf: &[u8], offset: &mut usize) -> Option { - if *offset + 4 > buf.len() { - return None; - } - let mut bytes = [0u8; 4]; - bytes.copy_from_slice(&buf[*offset..*offset + 4]); + if *offset + 4 > buf.len() { return None; } + let mut b = [0u8; 4]; + b.copy_from_slice(&buf[*offset..*offset + 4]); *offset += 4; - Some(f32::from_ne_bytes(bytes)) + Some(f32::from_ne_bytes(b)) } -fn write_i32(buf: &mut [u8], offset: &mut usize, value: i32) -> bool { - if *offset + 4 > buf.len() { - return false; - } - buf[*offset..*offset + 4].copy_from_slice(&value.to_ne_bytes()); +fn write_i32(buf: &mut [u8], offset: &mut usize, v: i32) -> bool { + if *offset + 4 > buf.len() { return false; } + buf[*offset..*offset + 4].copy_from_slice(&v.to_ne_bytes()); *offset += 4; true } -fn write_i64(buf: &mut [u8], offset: &mut usize, value: i64) -> bool { - if *offset + 8 > buf.len() { - return false; - } - buf[*offset..*offset + 8].copy_from_slice(&value.to_ne_bytes()); +fn write_i64(buf: &mut [u8], offset: &mut usize, v: i64) -> bool { + if *offset + 8 > buf.len() { return false; } + buf[*offset..*offset + 8].copy_from_slice(&v.to_ne_bytes()); *offset += 8; true } -fn write_f32(buf: &mut [u8], offset: &mut usize, value: f32) -> bool { - if *offset + 4 > buf.len() { - return false; - } - buf[*offset..*offset + 4].copy_from_slice(&value.to_ne_bytes()); +fn write_f32(buf: &mut [u8], offset: &mut usize, v: f32) -> bool { + if *offset + 4 > buf.len() { return false; } + buf[*offset..*offset + 4].copy_from_slice(&v.to_ne_bytes()); *offset += 4; true } -/// Calculate serialization size: -/// Header (32 bytes) + count * (8 bytes ID + dim*4 bytes vector) fn serialization_size(dimension: i32, count: usize) -> usize { 8 * 4 + count * (8 + (dimension as usize) * 4) } @@ -279,22 +278,19 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreate< max_degree: jint, build_list_size: jint, ) -> jlong { - match create_index_state(dimension, metric_type, index_type, max_degree, build_list_size) { - Ok(state) => match registry().lock() { - Ok(mut guard) => guard.insert(state), - Err(_) => { - let _ = - env.throw_new("java/lang/IllegalStateException", "DiskANN registry error"); - 0 - } - }, - Err(msg) => { - let _ = env.throw_new( - "java/lang/RuntimeException", - format!("Failed to create DiskANN index: {}", msg), - ); - 0 + let result = jni_catch_unwind(&mut env, 0i64, AssertUnwindSafe(|| -> jlong { + match create_index_state(dimension, metric_type, index_type, max_degree, build_list_size) { + Ok(state) => match registry().lock() { + Ok(mut guard) => guard.insert(state), + Err(_) => -1, + }, + Err(_) => -2, } + })); + match result { + -1 => { let _ = env.throw_new("java/lang/IllegalStateException", "DiskANN registry error"); 0 } + -2 => { let _ = env.throw_new("java/lang/RuntimeException", "Failed to create DiskANN index"); 0 } + v => v, } } @@ -304,11 +300,11 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexDestroy _class: JClass<'local>, handle: jlong, ) { - if let Ok(mut guard) = registry().lock() { - guard.indices.remove(&handle); - } else { - let _ = env.throw_new("java/lang/IllegalStateException", "DiskANN registry error"); - } + jni_catch_unwind(&mut env, (), AssertUnwindSafe(|| { + if let Ok(mut guard) = registry().lock() { + guard.indices.remove(&handle); + } + })); } #[no_mangle] @@ -317,21 +313,11 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexGetDime _class: JClass<'local>, handle: jlong, ) -> jint { - let arc = match get_index(handle) { - Some(a) => a, - None => { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); - return 0; - } - }; - let result = match arc.lock() { - Ok(state) => state.dimension, - Err(_) => { - let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); - 0 - } - }; - result + jni_catch_unwind(&mut env, 0, AssertUnwindSafe(|| { + get_index(handle) + .and_then(|arc| arc.lock().ok().map(|s| s.dimension)) + .unwrap_or(0) + })) } #[no_mangle] @@ -340,21 +326,11 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexGetCoun _class: JClass<'local>, handle: jlong, ) -> jlong { - let arc = match get_index(handle) { - Some(a) => a, - None => { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); - return 0; - } - }; - let result = match arc.lock() { - Ok(state) => state.raw_data.len() as jlong, - Err(_) => { - let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); - 0 - } - }; - result + jni_catch_unwind(&mut env, 0, AssertUnwindSafe(|| { + get_index(handle) + .and_then(|arc| arc.lock().ok().map(|s| s.raw_data.len() as jlong)) + .unwrap_or(0) + })) } #[no_mangle] @@ -363,21 +339,11 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexGetMetr _class: JClass<'local>, handle: jlong, ) -> jint { - let arc = match get_index(handle) { - Some(a) => a, - None => { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); - return 0; - } - }; - let result = match arc.lock() { - Ok(state) => state.metric_type, - Err(_) => { - let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); - 0 - } - }; - result + jni_catch_unwind(&mut env, 0, AssertUnwindSafe(|| { + get_index(handle) + .and_then(|arc| arc.lock().ok().map(|s| s.metric_type)) + .unwrap_or(0) + })) } #[no_mangle] @@ -424,7 +390,6 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexAddWith } }; - // SAFETY: Reinterpret byte buffers as typed slices. let vectors = unsafe { std::slice::from_raw_parts(vec_bytes.as_ptr() as *const f32, num * dimension) }; let ids = unsafe { std::slice::from_raw_parts(id_bytes.as_ptr() as *const i64, num) }; @@ -436,27 +401,35 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexAddWith let base = i * dimension; let vector = vectors[base..base + dimension].to_vec(); - // Assign a DiskANN-internal u32 ID. let int_id = state.next_id; state.next_id += 1; state.ext_to_int.insert(ext_id, int_id); state.int_to_ext.insert(int_id, ext_id); state.raw_data.push((ext_id, vector.clone())); - // Insert into the DiskANN graph index (async → block_on). - let result = state.runtime.block_on(state.index.insert( - strat, - &state.context, - &int_id, - vector.as_slice(), - )); - - if let Err(e) = result { - let _ = env.throw_new( - "java/lang/RuntimeException", - format!("DiskANN insert failed for id {}: {}", ext_id, e), - ); - return; + // catch_unwind around the DiskANN graph insert which may panic. + let idx_clone = Arc::clone(&state.index); + let ctx = &state.context; + let result = panic::catch_unwind(AssertUnwindSafe(|| { + state.runtime.block_on(idx_clone.insert(strat, ctx, &int_id, vector.as_slice())) + })); + + match result { + Ok(Ok(())) => {} + Ok(Err(e)) => { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!("DiskANN insert failed for id {}: {}", ext_id, e), + ); + return; + } + Err(_) => { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!("DiskANN insert panicked for id {}", ext_id), + ); + return; + } } } } @@ -468,11 +441,12 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexBuild<' handle: jlong, _build_list_size: jint, ) { - // DiskANN builds the Vamana graph incrementally during insert. - // This function is a no-op because the graph is already built. - if get_index(handle).is_none() { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); - } + jni_catch_unwind(&mut env, (), AssertUnwindSafe(|| { + if get_index(handle).is_none() { + // Will be caught below. + panic!("Invalid index handle"); + } + })); } #[no_mangle] @@ -498,7 +472,7 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearch< } }; - // Read query vectors into owned Vec (releases borrow on env). + // Read query vectors into owned Vec. let query: Vec = { let query_elements = match unsafe { env.get_array_elements(&query_vectors, ReleaseMode::NoCopyBack) } { @@ -527,16 +501,12 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearch< let mut result_distances = vec![f32::MAX; total_results]; let mut result_labels = vec![-1i64; total_results]; - // If the index has no user vectors, return empty results. - if state.raw_data.is_empty() { - // Leave default values (distances=MAX, labels=-1). - } else { + if !state.raw_data.is_empty() { let strat = test_provider::Strategy::new(); for qi in 0..num { let query_vec = &query[qi * dimension..(qi + 1) * dimension]; - // Request k+1 results to account for the start point being returned. let search_k = top_k + 1; let l_value = std::cmp::max(search_list_size as usize, search_k); @@ -552,26 +522,38 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearch< }; let mut neighbors = vec![Neighbor::::default(); search_k]; - let search_result = state.runtime.block_on(state.index.search( - &strat, - &state.context, - query_vec, - ¶ms, - &mut BackInserter::new(&mut neighbors), - )); + + // catch_unwind around graph search. + let idx_clone = Arc::clone(&state.index); + let ctx = &state.context; + let search_result = panic::catch_unwind(AssertUnwindSafe(|| { + state.runtime.block_on(idx_clone.search( + &strat, + ctx, + query_vec, + ¶ms, + &mut BackInserter::new(&mut neighbors), + )) + })); let stats = match search_result { - Ok(s) => s, - Err(e) => { + Ok(Ok(s)) => s, + Ok(Err(e)) => { let _ = env.throw_new( "java/lang/RuntimeException", format!("DiskANN search failed: {}", e), ); return; } + Err(_) => { + let _ = env.throw_new( + "java/lang/RuntimeException", + "DiskANN search panicked", + ); + return; + } }; - // Collect results, filtering out the start point. let result_count = stats.result_count as usize; let mut count = 0; for ri in 0..result_count { @@ -580,7 +562,7 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearch< } let neighbor = &neighbors[ri]; if neighbor.id == START_POINT_ID { - continue; // Skip the graph entry point. + continue; } let idx = qi * top_k + count; result_labels[idx] = @@ -591,10 +573,9 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearch< } } - // Release the index lock before writing results to JNI arrays. drop(state); - // Write distances back to the Java array. + // Write distances back. { let mut dist_elements = match unsafe { env.get_array_elements(&distances, ReleaseMode::CopyBack) } { @@ -610,7 +591,7 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearch< } } - // Write labels back to the Java array. + // Write labels back. { let mut label_elements = match unsafe { env.get_array_elements(&labels, ReleaseMode::CopyBack) } { @@ -695,21 +676,15 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSeriali _class: JClass<'local>, handle: jlong, ) -> jlong { - let arc = match get_index(handle) { - Some(a) => a, - None => { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); - return 0; - } - }; - let result = match arc.lock() { - Ok(state) => serialization_size(state.dimension, state.raw_data.len()) as jlong, - Err(_) => { - let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); - 0 - } - }; - result + jni_catch_unwind(&mut env, 0, AssertUnwindSafe(|| { + get_index(handle) + .and_then(|arc| { + arc.lock() + .ok() + .map(|s| serialization_size(s.dimension, s.raw_data.len()) as jlong) + }) + .unwrap_or(0) + })) } #[no_mangle] @@ -729,13 +704,9 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexDeseria let mut offset = 0usize; - // Read and validate header. let magic = match read_i32(&bytes, &mut offset) { Some(v) => v, - None => { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Data too short"); - return 0; - } + None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Data too short"); return 0; } }; if magic != MAGIC { let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid magic number"); @@ -744,69 +715,49 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexDeseria let version = match read_i32(&bytes, &mut offset) { Some(v) => v, - None => { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Data too short"); - return 0; - } + None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Data too short"); return 0; } }; let dimension = read_i32(&bytes, &mut offset).unwrap_or(0); let metric_type = read_i32(&bytes, &mut offset).unwrap_or(METRIC_L2); let index_type = read_i32(&bytes, &mut offset).unwrap_or(0); - // Version 2 includes max_degree and build_list_size; version 1 uses defaults. let (max_degree, build_list_size, count) = if version >= 2 { let md = read_i32(&bytes, &mut offset).unwrap_or(64); let bls = read_i32(&bytes, &mut offset).unwrap_or(100); let cnt = read_i32(&bytes, &mut offset).unwrap_or(0) as usize; (md, bls, cnt) } else if version == 1 { - // Legacy format: no max_degree/build_list_size fields. let cnt = read_i32(&bytes, &mut offset).unwrap_or(0) as usize; (64, 100, cnt) } else { - let _ = env.throw_new( - "java/lang/IllegalArgumentException", - format!("Unsupported version: {}", version), - ); + let _ = env.throw_new("java/lang/IllegalArgumentException", format!("Unsupported version: {}", version)); return 0; }; - // Read vector data. let dim = dimension as usize; let mut entries: Vec<(i64, Vec)> = Vec::with_capacity(count); for _ in 0..count { let id = match read_i64(&bytes, &mut offset) { Some(v) => v, - None => { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated data"); - return 0; - } + None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated data"); return 0; } }; let mut vector = Vec::with_capacity(dim); for _ in 0..dim { let v = match read_f32(&bytes, &mut offset) { Some(val) => val, - None => { - let _ = - env.throw_new("java/lang/IllegalArgumentException", "Truncated data"); - return 0; - } + None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated data"); return 0; } }; vector.push(v); } entries.push((id, vector)); } - // Create a new DiskANN index and re-insert all vectors to rebuild the graph. let mut state = match create_index_state(dimension, metric_type, index_type, max_degree, build_list_size) { Ok(s) => s, Err(msg) => { - let _ = env.throw_new( - "java/lang/RuntimeException", - format!("Failed to create index during deserialization: {}", msg), - ); + let _ = env.throw_new("java/lang/RuntimeException", format!("Failed to create index during deserialization: {}", msg)); return 0; } }; @@ -819,22 +770,22 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexDeseria state.int_to_ext.insert(int_id, *ext_id); state.raw_data.push((*ext_id, vector.clone())); - let result = state.runtime.block_on(state.index.insert( - strat, - &state.context, - &int_id, - vector.as_slice(), - )); - - if let Err(e) = result { - let _ = env.throw_new( - "java/lang/RuntimeException", - format!( - "Failed to re-insert vector {} during deserialization: {}", - ext_id, e - ), - ); - return 0; + let idx_clone = Arc::clone(&state.index); + let ctx = &state.context; + let result = panic::catch_unwind(AssertUnwindSafe(|| { + state.runtime.block_on(idx_clone.insert(strat, ctx, &int_id, vector.as_slice())) + })); + + match result { + Ok(Ok(())) => {} + Ok(Err(e)) => { + let _ = env.throw_new("java/lang/RuntimeException", format!("Deserialization insert failed for id {}: {}", ext_id, e)); + return 0; + } + Err(_) => { + let _ = env.throw_new("java/lang/RuntimeException", format!("Deserialization insert panicked for id {}", ext_id)); + return 0; + } } } diff --git a/paimon-diskann/paimon-diskann-jni/src/main/resources/darwin/aarch64/libpaimon_diskann_jni.dylib b/paimon-diskann/paimon-diskann-jni/src/main/resources/darwin/aarch64/libpaimon_diskann_jni.dylib index 8cf7312b036a9420095d62ca83e26962cd5c0e57..556f8cdbc79a062fa831a8774421652ad223d6aa 100755 GIT binary patch delta 229257 zcmcG12Ut``*Z-Ya%7O?8%hD7nii(j$tYBNv*kXx@Jt|6~sIiwsjVy>!EK#qsdW}h} zi3-MqSdwTg7!y-pgE74r(3mWm*hR4M|DCzJvJ&3!{oe0+{yfj+-sxv%&N*|=Z|2@D zI@tNj@y@GP^P!pQn(hIX=|+YAelW;<6J$u{$xEf zg}FFp3>Q0DWMHE|^wf@tdMd(N)K_&Hhzca8H}sOR9iP_|5!R*;_X-x9nTP1fBZFSm z^bGHJ(lbz4uX{a@3>f;Yo;od+$>gEmiqH8Oc4cqA;VaIsY!UN3%3G`s_EB2bQIyPo z`6$KvxPnLJNxe4ZTW>{P6jy@hG$Px*_;Vj+yHHu6DE2h^H@&)R*qfONeGU}f`pmQF zOun#tz8M{Y%*{~99~U)1e+_U^*s{=1DQu{zExZzh-Pkl>!ZyXExLGRM&8*W4mF(2# z*NM-^wB-}miO%nsO;4{=Z0R2=>b?2N`8WE1xU>0Y#iaSC@*wfD*VM4iNX?v=T)9(G z_N8YfSF)=MWF76-Y5mfenJ26hN#30sMN_sZj%*W=&CyhSs(fJ`_53Eqwr&D5anm~O z43+a;r>%SzcjG#7-`5<+seG(OZzisq)Vph`JX1eNBO=|@`m3~lgP7U-nv8c-KiH&A z8N|YP*;+BGj+viZtF6pL@~@N}ZBVyw(%#BsZTZ2i;$)rpuw#fWF$5O8t?0F8IcoiF zr4~PdnZ+YMA^dHL84sN3t>Q>sbKE#8g#O6kL;sxPk}eCe2ZP>ZlgQc=#|NyXQsyTpl`}OGbe0%Q z1r;IOwj)?vsN0>lUMnK@M!Pj%>zt&?zJURevbO3e{Bxto+8fIQ)@ql+4Pl~auRr(1 z$k2GRwKU%Rj!irAEQ{dRZKADTTYmYFnBte{cICSd>xx}|4fu%F;&ZdG%Vzbs!rXNe8wu1`{hs5H*Xx?j+C1h~dF;{I@k?aqwV1X_Pn<9Kp{o7x#nBVIQnPPlY~D zC9A6EY5y9cE9q?{R|YEwh_>fmRczsCWbeYjf)DOj1(&q4RDQ7>#b2>h8o#aDSF)+P z_?9&yDp6wgT(J3l5fHV(J03|Sr27KIyHODy zcR|Lk0P#~)0`Kv@@QqIPqPo-$s8_t{7v>gHP`F&oiH_#;4vP)Z;{tv>uh=#PJ8Lx+ zM55N1DG>({ZO^sj8|&)y(HO2k)g1!3Eh#{3jL9+&w<@;sU`wUhUDdo2S0}%N`0QM!PV0mFaX0nUV$tsrv-b`e?WRuC#FR(OeA7!J zeyy3m{F2y+L_vu$2K6aT9C@T4Z+>0)wm8D?ToWf-#MCVbRZ9$^1tdOqe8v41KN);5 zt(;wl>SBamd7x3pOCQcO2=os*o25yCriCoOk zScCeRChpHL^RE{P^UQhtRhUT$Q+S>x-b#oL2w0)mG&wA0IV_$lMB7;rYg)Pc7KMP6 zV31-ic(*PW<7fSykdCQ#sOG{HhiZPcF1Ojdpn0w{+xN757=j9Z#q%e4o;+J5w2rlY z<;!h9<02t&=)OWF+e7^faSKos(_g5h2dEz-Jl7dU`bmUmJHwG)>S2VZyM#|5JQ-mr zTwLnZLWNW?;>SDUlMPCTG}(ZquDTluW1JNfK`A3V(j~kH;bG1&DtgOT1hk2^u19nq zXEdtWf^biSsa@2sR}US(67eaBFK?n$$|fUy)TM~)^q{P#`Xa)KE@3Lu#u-NbWeB%) zhSLq|Vqb2xwQxqDk%h?63}I?~%Y#k_s#6gkoYyDa)8!G3;89GoA) zC(jf=%@5-t%S8G7ar|VN81=$%Kd6Ah6blv=NZ`ggTzvAv0{-R_(S1R*=LW3a!(&3l z)CHXkag2$#7c}6X2yt>jTUdyDJ+kI8KKwb*vXvjXBx19Z`0vYwZxvRrdsz_0n9>)}mX!a_m`RLjafoErH&ehr9FTtrcXub;{M{7$Blds6!DUI zy!bis?vm*6#>iI!f=C~S({sGk#f8TGoyDf^iG0UjB0eu(7<>5eWq*l@dEGsCqERCn z&FGQH!w!m#J>vMJzeL2+?)=;(F>YzJ=gZKwi5SgQOFQxC`{Lx%h=8Lo2rgh|Z$K&1 zdNFFXxW6MUKT_DLqFx8qet@IxK4-+)7Ins)ZH$Y$_``p#;x>a-cwNLP1L#FQ(Wqs!dt&668iP#Q~Qgoeo6eJ z3&MAGEU%L%)pq*oj(p5&v2k^Dz{N$*g(^hyWyvCu)qfuUaFOVzn!OJpHT}4vzPM6M zQ5ObC(U6*&7LAaC{vP73Cld^19#}Ni2JwZ9&i zK+9@nu9#wr2n1cLQzAP#7PF9oo$g$0vNaad28Qr6x#En?9QH#l$zJ_}6PKzqT0_^S zfJm&N>OiWlACQL==nxZp#fprtJ!;>r1y_9_mnBTAQ?f zn9uJEVHsrf%3p-#P86;;x&MNQouX|0RNj8282<7azG0!XFl>~u77JtLLTO>-QF1L7 z#zMr?=p-lWqqB9sv@o8R(Qaz;3X!$J>@^KR`@O2*z2dbE;o)^RU}UelD4kZ`DkP*B zawV+82ARh}Da}MTlP^`Z-c0UKb{+Md%cAou(Y*if!gsit_xfGTdu1i*gVl*$TZj*a zALe_X7AId#;uXt<_iNX9xmDbMZ6TkXBjQKK@%Nt)Z*A<(hvta#jq$usj)>jVohRjp zDVxmZ);Wr;i=Q%`8I`oW4#}0u7{C}S#D-0=+_%4&GAf1VEEgV|-%a@=Td{S=D8k4n zRN&)D7IOG_Wt`jIC*#5)oeSZBk#vrKO0m6%o{;1%eoEZmJlI%-YJPxFYlY#SAtEyU z5#Dc_=$D?vH@k^t>0$hb3u1eE(xaIMa@!)mNtXtApeomL)Vc%ldvO8(GD!nFK}n2v zVS4hz2HoTYGK${zgPaj(i+I+B7f;Q)^614e^Qe8ls6&RHcT!J>AS|w<6 z)aoRSK%0HCwrbF3K57GKv)x(^+Pq@bL7TPU9)LEhtU743#47%HdYRdM>OY;VCQGH0 zHP`?rYn%uf=g&X>T{>FLN*{K#(#jrowB)>|o7Dtx4mnyfY^krtBkX)`?|eqSHkeCt zv=UJdIa))Q?r3dB9;c)A8W;B)&-2WOD3PyqdVEW+>=7QBu{`y6>1%bLO1>8NE*C## zh6ll`+5!8*)vM~B2d^rgv)`phH7yG=?N~&|;n~HbT5W9U36H8JzkZIqsS&;Yc-Whg zT__4ulbP;K$*|Hu?S!!Nxrg%^`MTkmyea6w;*{*B54@=pRZLi$we&4Sy6h(0sohMv zQ(F*K0(UA1O_Mv-SCQ`2;mOjSICD)?t zovU_2URCcK{+D#2#BHgLp3#2>MZGl|dI#VLnHqRw+r#g!#r}*>l&cda7 zOZZRq=iAT1t%?wqsqVfmuGRMYOjNb_(eNIiZ}P21FP6U53t(&bR=#x8eXF)s>03op zjP6^_gQex_Tg`&e2H$GIEa_YISS;4eoWffzk-pX8`49V6d*;KpimB`3SdFcvp{mOgdG|&s95Bd3YwLD&!flcy_FH-hF0UjEfW(IaQc-I6vX) zEX0vhCBsEc)M*HlQ{_ybi!eD=GMpWuW+F^Zl`}jYVOOV0##qj+3f@#mhV-V6&z0WP+#lghjpM!Uh!Zbl@gdK_foj5a--&;FMZC2j zg3rj6uG5}Hy6faV9?gywzgsPQ!Wrp4t(q_0r;o=;_i5tuM2GG^r9V&Z6CZh3tjpOF zTpabOmzqJYlhXKs>$LYrxK3F-VVrcE3WiI!sduh)oAx{|-KO-hqV1xu8#aPPTASOn z+~GDgd2tw@Zxiv|Hi-I*U*!SMk&EH&^Bk#vyt1XM)ESLAU8UCa9N}nqj5}PVmi)t?rJKa&NH^)u z>}ogZxD{>^l-Kh5F{=O?)pV7XIb5Y(4p+&#yf3$}k*?B-+2X?T_?UE*s_81#EY)E} zUw&W>T%`nl_pEf2dVl+{o0Ru`wVSlz*V^5rJ5Rz*8qSZMg^Sd-H^0SfbKnqFdIAW6 z;d7^sr4s55kbRI^iY4VX5V|QymvN5O=ol8l8;Eyxr2cqPI#P8)(JDDo18zZR`X=+= zW{JhCdh@U@@TB6z-Btd4>QB;}diS{p-jq+a^rqI$a;{UlH?@40^rj}ylHSyKq>?xF zSQfmgCAwzT9jV(lMd#84gRPuNN2NMQR zkD6oZYa{sVnY2zeoQhoJK@}pGs|OW^N&4(2 z6Mkty+>k*o6d#0bnzgC?{1kXlvD_UysU-*`?v7Lez}-zpaq*AMquH3K$)i{r$X%n*O9pTW;9kY3Z{Go&HW zg>u#6H6_lFhD0k$uEmgOfp{_`{(4Lwk-n3pArT{^-PFJ4lh?$BeceOM=5w@_Y3#AC)FOrET-2r?h3N z^pql}OHV1IgY=XP)1{|$Ynt?w&QFt`(r?rL=_w6(L3&CDI!jOKk@?b7Qjw87B@6rw z@|04@v2}P#HXw1ec}f95sGXit%Rb~O@k!If;4N+W;n&2g>lQ2nNIlP>3~Sor#It4x9&q$lZi&@*36g@OFf|BK%N37&8L#H~^v7i}0)0*k~Kh zQ#_mz*1=L_Y~{>|@EU|4b>t(fOE!$v-HLb|U(hQ|Z=&WQ+}IgL`l4&%;2Zt;@L~zXgDZ*n%Z!RW^)E$F)qeM}#BZ<%bM*Oj3 zVc7O55S~XJn;O{kbshlNJVh+r*_}U@BtG96R)5!c5IEW~_Zig$=Ag8&Y%}$nkB8cr zV>aT6IXc5g2n8wMEi@IZBL5z}SmUTP?<@4$rP9nPd&lN`Gv51oF{NOR&Mh?<+~;5tvKKNmznKvz$Ya@(4P8Utb1!&OvSVM#%dXs zuBJ>$^u*v&aw{x`H2woFGwtng&rpdlSf3(x!3oA7!tg}$b~5-P1Tn# zi-CZ|~VG?@d3MwSbaeD3qDm;PI+CYWJsT4qk@y|$5 zq2sd>REU@>L51d2sFsr;jb))~P~orH5>zlhOQnMSDb)p3IE(>#_7W2p>pspaCrMP{ z$|NUkgesh$L^M>R3U8u1pbFnlszw#QoTQ@)CniZ$p=gqhD!e~Qq6%$C{1a6eI$NR& zOD>B``{wZSVnP&BhyUwgMBzo4L9Sc35cNW(sJWVPiAh93hBvoVMH z+`R2@7Is`Mo43hO{Iq%d0~t9ko43!+lAE_rzwgfziY3&rWg-?r2h^b9*$Fi?rOn$@ zzH;-n7)Kx)6-&5b-q#Pq4YGfckEt^NkoxOt4Lf!TDk0o54bRUzpOJ4io(XQyMWE9I zxZ&1C2{-taB3+IG;D$4QGjX(8i@>iDR|4a(JK83^A?q)UPP4WA(l7})T%O>xrgXsJ z_X!ekcmp>84x5l#o1O6T1ZgJ-O0LCDcnR@jC-iBm+X=hJN;@G}M!TtfV1Qx&cK!qj zILw|PZNRCN2ph2dOwsRfKkoUJ*m(E||Nf*HbtJ|WWcdC9=x^~Ow__g$8@H=xNQ_}O zm>d{G7rN;f!}v)OV;D#=I>xXI7Mm-^@CFP=U<_}iON`<98RF!JQ~1*}CB|?Gn$&gw z_S|^dzZI*GySw5HYs!E#TuO+9zoR^2q9t~9N_H0sH9QZ%i}r61eN_!L9KbU{4ZQ}4 z4?l{v?gHNV9xf-;fEiz{SX!!YAda908HSa*1!003oaqG!6VxEXC9Tw#5hkd?8QzMp zE7Tz4H@8+-A>IXQ=#VB8l=kXUBoN*p!`ZFW7ZE1B!5LnOFyRf(ik4jvTR)Dr&O$Wd z4bEy7Anb}a$oQfJbt2-Oc!P{9X{$biIKmtBvI*)Kgb8nOhN%qU4bJ@I5GK4qh9l$E zu@@Y8gN#5UqmaQBZ;Fmt;>*wd`3GM}z~Mrs1RQ$xmw>~KXC&ZI@G))Q`but7n#=9md4Edq zAumJj-VTRZ=h(gF9`GlCKx~^T5r}isbOfU9Kyl>cR=(kLi9QsKm*_)zsze`Nd722* z(FgTui9W2f1AW*S7KJT@sLXn54D@40deQ>`;`fi`-tE3qX*?rUf)GDY7&5#I0rwkMU0^u=Pn7y$fp~MEL?9Rvoe0FOv3QPG_Cfy^8r%4aN54$Z=U0E;>YRx9HG(%kCp!OX4rmN0)fHz5?xf=ky-_FO3=7Zc zID`Kf@!^@?{E6nm_cv@HosbyApBWD@hEC5)jNy}X=hCEO42RMs#;`G6Vhrn%N*KeK ziE_tQN`uReZS%7dX!!W74m6BPmy66$6rx2Y3WWeP{B=$O4LhVUWKg%@PC!G!BndP$ zsnLG@bO|(U7$Jd%r!aTt62jJD!-O_$OGZBoGJHFlHf#f;$%k}78WxpGq~YLbi8Sm( zAy=efdIZP=((v_Ai8O2#9_u~5^)F~EfI(Qm@7gYCZk_X{|(~s`~-!0POi4u31gJZO?C#F%}T5yMnqoh6YG$q$!PmDr5*%J*K z=;M<$MA{Q+GTKdTkV&|MPagznw{|AAy?H}Oco{&yng|3Wq>wgj{VohLr$7@q5r}jY zq7B=ZzK{q+^CF2rG%k`GwuzWU+OU1%;$gnZB4LPc#!DFD>k$%$7%)=85Vj@~hG;!f z!VpbIN*KaCQo<1aBmW6QEXZsMb_15-fgtldUZHY#=h=Vvv&uymX4FD zZsSD6wS)Z8-^5SXfN_l%x37iq1!=;3UF~v(j2ngn8-!gU z1{n_{<{07$VsM5(N7xl%knykw-a|Yg432OIwFnWe5QAPpXLUE?31ZMQc2wU*m>>pc z`kpg_-|WX5{3cRw#(UKJjoFHR6^m~sr924_D08p_Vo0s_;CmyUAclMl65$0M31Ubw zsF`UFh`~dL81{}6ygWLf6SBJ=piCGc+Lp)ibHD5zRUW{&*E3?qtrr9T!3m%KBD^c& z`JUk-sUpm6*KnW)k5!t)vlWrN+hDP>Vle-D3~2k&+Fjr@h#N+pI6T99>I_}*U}6F$s=AwZE3 zhG4{Clfg0=%h+W+&-&Q~L)h9=VhA^W(J_Qaj*G!nN%4n%1~)2^-#90kxG@=k*B!)r zqD5CQVc{Tx{t}^qC5nF*}vxWW?N|m?4}jbH1-L z7xik>JecW`sYs!-?>BIkH9bX9pBtd(oXlu7G8KfNr=R@9Y$tGO(?D1J(puWfs?pwH zM|<<5way+anm;jA>-;C~DMPiz2tS^PbgJEsadWn8akLzz=xL=$Ta8v%;kwcPNdCR~ zEdO3tMZFG66$?{;(Ej#dF};hC&5A4-yu^l`Z~`1Nr$j1hY*WRAcXmuQ<9}_UnDVK^ zHL7eKtqu1CiEcx+EKe5KsB(x-#!e#9B`;0VPNwV&EVkrpk)duT3oy!k3vnKuj!ZcR z_>UoIIT92(-hy$)(_|_*X6X!n_B19UJJc=Yk#8OCE&6AB2kcB~4aS(xa2EAJyV{k( z22*4a8gf=vv-0iE%DwkEDj)N+w#Sntd3@A|nWDbezAi9KH*ux16~>mv zwS6C^O?}!B#C?ZoN80d+SPw^!_M;LzIEf0@uPm!vbmYn+uy*N7Zr0Y`3yy;w30v|e zO3_F<=jweq}7$E$(#;<&*9##4T?-p6JCmt(OlAcDt|* zDz&gKi{z!-w8cKi`UAyW33ZF)Uv1M)B8H#XrgipZ(RB_Zg$OXN=&h~tVWAE8AtL=O zSLZwlfnWGUg>_+_iq>*`Oa+skVA`sa7@y?9+LXF%EMMuU|HTLOzu>5UrlbCeHR|VM zPy`KIuMN_NaEr6YXtT1P9``EZp7?>;dgF4H3KGi9c~~Xrm?>BGXGSZkl2cV-0lPG4 z54xdRJc#ZZit0}9hZWW7L5g;1cl7b{=L#qJ2{kx89Sm(dJ0puCmA!j zSxg{&b3MF2`Aa`7Vi-4vHFKmVm}oMC3ckZ&y$AvqeNS>?=EvUF?$=`x0dv6_vsYRw z+8C65n?F?4K3Q6P02|DwzOKv1s{yR3_vqIZTV3?L_xIY#02XcTM%)>#s8~!;xvAhK zRNnD>%_9&jUDr{;W3Owz;|y(mqfwUfj6@!v@3n|4tUG@>P4hKD2v?=pJ>oVu4IL>NWN73w6^euik3c!Cr-{`!=k;Qys$Auib8#raZg}Zw?M#p%I$$2s z&O%KY7g_R2HQd{%y=7vdO$wtnn`ntH%=pr=MB~u76Y?MHI7hhYRqYZoHz@3VELu&O zJE0;DYkBV|e>EA;RGalxX42AIL9!Er*z&f1ASnRkDUGsI8R{_NITGV{6BAzA)lKDy z_dp(!=u&k3EV^;9pLRP4!xPm}i!ig;h~0p($q*r7DnXgCcd9hXzaDec<2&tS8V}Q6 zFtbMdT2F1O856QGS-Wj!M|i*e+QDEJ)|S1?Y_InOGlN#3Y-%2)v0Yu+I&$!#^tX%@R7g#Y=qHa3LK3)nInYCG4c+HsW_ zd{o~#+NBT{)+LiGtBpA=)UtElYS|=KX`jj}o1#S{PMtQxa$s-blV4UByFPey z?~7VOeHP~xd=C?Z-Yw9csn16E9*0*~L35YusU8`rovhDd>fsHZ8yGt~u9932b^QR1 zH()E{BGEA?tC;azVpR53600B|v5E=1RFxRp!Zht*12&LzuK7i<{oXHO7Kud`bm zusA-qgLXTDh4r4|uh=-GQL+x|_!n5mq-3wjGt8=hJ$p3=R?*Q!!?#33ND^G*z0lK4 zPaA_{_b@HKO{oF-<)Jtr~1I*Vz&Xllqm-fjxqt;)4*r1oCte zu#0-PN_&uJbaLgz$|{W(((@1U_^8)!S7~0(dO-G*jy%6O^I&jS->LGEV$cNQ@!e^r zZCc_MW;OVpDm8E{`3meVyftDl6r3}-75r{+FF0@TDDZ?{##_kJoF0R|x58NTK|>{; zy?2**2H<)ES0JvQxcqVT!c`AfcU-=>df@WI)eV;quE%#v=}GohOFZlDwi=Het+4wj z`(@-9Z&g9K#7Eipba&wK5QhB_znu>8+v207IYloMrd-WokhQ%gKk(RdiHfw;nPJ&7yCihl!eh2rXm%Z#f(u3%h!aG7xR#TB&M0-0Cv=5MLN zTm1`DLepbYly*)ca=dinh=`6L!Z~WkFlwbZj{wfuSMr3Vk_^3in0BkT@{WaDl{Bk| z@=ht<%whxte-xr?R7)sW9YbVLFeTZkAMxze^2T5{LvLxETVTMJAblO?*$+LGW%rr& zvg@nMrcha2f4Y=S2SHSJ3Cad~E7@O@YOFH&YJPccp;p;Vi?_OU57Zdcnhd9Vt3&td z;nBs_>JNTm>-6hYZC*pZVTez^?%9sutzHDlEErxzlghSAvu z^m%5%aEvXnGQ&lQv^7^BY#i=nWlZidknTFZ#JNEgx>y81$cML!(J0k%1FKgXA4{b@ zS5Y_fF5lUkq-YI2J(>+k#yCDd+>%cT1S>F5PZJb_>+@C^?-eA zj4_y*a`csh{1o_zWGv9jm2TMYB(<62VW}*=QeCDS%4|g1Yq*>%^M0t#EYw5sRmTmd zwBqZ`WGz{){b>l>#(JySA0DkJf&%aLII|5x%dWDYL$72+gMOw= zFZ0|KcGOV6eg41>`EVgy3z$m}#1Q$ip)b0ij(hX_;j0>b^UH_y$*&mwL_Vl!qmpw6 z#gJ;oB^%y~h8N<6y{?M-cPGNCm4dT)CN`f{R=L*^M!eSHx_hYA zv5prE$JEiRL#t;cjcld{SC``_z`VO=GhMup- zCa!Cv>36HzoJaf`#HZiYd2}cEw*+z2s55<8E$P%zod9iY01IeEZN{QavXVw%bSRxh zEEi=TV%1zHtNt&R17BiQm*t?OmgV3F$c1E8R7?&2$K{}Eq0Xuw!yB!+9FW^xir$Uh<})ASAd&qW7;KM|9hs*K^c(!wPf1vmdct; zI~VEyBhxI+5 zt)=CEW?Jw6mT6l$nD$oRe=_Yw#QlF`TIkw;*f8VE9s2blru_tD(CpI!_3xQ>CYG++ zn07Bp*Toe0;Bqo;BQUKIVZ=Yov`3Io8`B<0`R|!_FUma7wB~7?NrfpUW(>D13$+Q8rFzw4XYh&8VJ~cFLpV~C-8(73@W7_j5-33$78JCl32g6=V zLYU&^H;^m_)2S-B`jv3=ON1f+Jih(`=c(W#G7`c@rqRvjO66Xx)SF>4SUi+{=aZRj zC`^Wsu3oB9Yu22FTW=iaiDmu0^Nskl0qo@~&WMsvUq^$vNF^eHw1-_9)C| z*KX2nm5v9}yQBZ)I~xr;uC^r(?*zgc`4UY{1S`op{loM+*M~i*GXvS11cVWhh$K8~)2Zdso8s<_dGh;v0 zwNj*i;gUWkNJ)DIHe1u{3m(*GK-`aJW#78sigv%Cot{?U8uqHo*ESi97tf+X1)6T> zSgzVgY16&Z+tZYH;E4C_wN!t{`8dyuFrsOM%dHHZd{pE?C%GFyqDVy!5q-8F#RM;O? zuof%3qQjN6hAjAE9_}SmdA=EjXDnElPVVHu*sPD|$ok5@JK#zJeFUKtaOGL# z35G7-8j*b9?0DrsPdBCB&Iop(^ih06Xd&;f-oXnUW4bH0vxq1iZ#l3T8|b$lS8R6> zriOf68sgMYsBUIaL;3+#xcG2?siAvcY5n3^sI@pM+g;V-@xg=?Kz>2>mt)5~KX@3+ z&+~-$*uXgkrNn1ZeVyxG+LmO-AS#{*Zh%BAJb>7K6=y zS4r{ej&c)GpDQyY*OhW>t^C5mY!i@}34x{=uHxp4+v@0?Dfn&%8LK4j#POuB5vQE3j;o7&a{MCQ)ni;)9t-6UeP;R& z&(3)AsR_WZk2KmyI)*xmuE{#AP<%?FKqc82xR>?HuB71YathOgT!w6M`9a&+)1a2QRboH%b(;)w)%JBzkzu7`QVHl|*KA4k~Y zI_QBCH0JEd<=*-(Cso!7Wyo=O9F=v$MF%^aYwBDWoF!eEns%*9>x8rvTo2%_dQSiQ zI&y6`gjE!2%*=u(ZO&zUUfjxN1g4Y~{m8N?zEs|sf}tAWkB4mluT)75|N zi!wWqwjGy~1CGGZ*orXXwW)L68e4B;qoe{GCfBiXqFYv8`F{@4EvqNN2!v^tfX`=o zEBov*?rKQ}S4+WWDG-z)(KH#4OJ0uhR?_a~x~Vz2$-8q5%5M7y(5Bl^&$iTjW;E&& zpN`e!99K{G!k8j&T$p0D-5Fu2oCC#^Zb<$yGELdnx(i-K1l!C6%9Dw;n)WV_4aCBY z1~W6NO`!{#a^OOSdt%woru?4Xwb*hQ9+KFz5IVBuZbX3CjPBL89MWyqtPn8Gav%=| z-Gj!SRMLvja3~0|d!l_HvC_x&)n>2`?M;I6AFA|+2-WUVDVaIPT`HAk4$?DFX$hug4DNP} zrnWhd1$iIm>R<4ntGWj2hD|UGEibD@G_`uY5`?bUdtgPr2aC+0>@Nc{XuqDd=O{i{ z2?Hl>R$$J7K?Y^t9&}Z=DfEv{?LfE)s|bm28TN(BS}B!Xa9AxDW%dUsrzG{3MiM`w z&Fvk~FPS`Oxv!#FO}cI1iA36i2rB6R1+$Igim3{f2#g0+NR@qfuPV(BI~Z&2J}>{m z057H-AmZ&v55l*TjrSIuwlihF{qh1yFqXWuHeA`a1jGeH_YmICaDM^e5~%WQ)Kk{O z3TnfZ$~qW)NeI61=Om;B2e#wNwsqSJa-Ef+`HNaqY4V*jTB& zcn!U|4}%%Qe-VJKm3duMUbfs$$6i3`c*sx}jEZ8sxUC}$bCN$Ay%->`7?DVv<*@I7 zh+x#BlCgJSINmY)94r!S?Cvy#R_w&90>-F1YQBM=vX;e!=%xtsQCk!Hoes^vVGYl> z8$49|_n7~#SjfOl*C7tK@Rn*txTSK+3(Q7&om}bU#&gI^Z2nloYMH_?r%_(2HvH}x%O=FR_!|!xpdRQR|Z45DMtE+!Kdg(raR^gQc;*< z@liAV(c2+I^Y4Cvkr?8ae>Ybl2AiIq04FF)X%|vQX^lOoudxU9_5S-ed^A7~LEjU+@OgCS}IGZ>p+lo$ObC`7hjidFFGvUcIn{<)flI^bh8|`H^&!+=XN~wod?z-kWJs`EyaY|L5&v# z_W}M7R@MpwnY(qHLG1xVr@RR&ZHEinAoTV&O(I#4T`|KcU{6{?J_Y;s#26Zr0W~mv z-O7Q3NJ|OWH}xiDidOGoonZceF#8YyyI52yO>!LAyL1~@A=s%y$S%ObYL$msv9i`& zRq31IR#2arkhkG4^r|UX4?Xx=P9nbG>J37A=Xj}B?sc&!1T2pHUBKd7@qAH(M>}`I zq4c@U4mE?EG=#=EhOZP8A+#!ohrmy)gVFR>tPfGQ1*rb4f&$@(%z}-+f1|o6dI4!(T`}6hM)xi zW#s}81Gf{MEuJS`(PFrw)(69L4vFNx2)uCfV0at^g@GW_DhgY$$946fBfnS$3h5LY zNl+H1g&e-i`8bw~Q7JzmU+_*Hi|6Zjw!E!7kC_=t;&X^=typpP7!{=86T$i}zOmqO zcn|b?haC%ZYYg#&bm+JaNJ~H(aq@4dF#Y%H3L`NxKT#u}Rmb&!*GhZEw1Ahv96D?8 zE+~O$uEQ%&k%+rRN=ObFt3ziRO}CukF*w!cNk(#g>As%zv{WudA#x&Wt8gLG=wvFv zm$j`NvJ==rFEL2O~|F^o^4uyf8gL*$4N{V?CX5&twU3cDlR9nZsX zjmA}g-V!c)mp&E0#4o=mNyK&Kfa?@Frw0o>&9@35v^WDF;c_UXzH-ighgjSTcW6O} z+^5srqtREh^fgd~JQ9he?YBj>~sl)8e|YFyk8?m$eGc2aG1XA_I83npoXh?{(mgI~Z*^hjAuCYH9HNc5#Hl+8+IH08^mqoi z)M4=jgCPAX=$wkOq!Pc{sUzFpeaz^jpfOfu-4mxpB343M4M<~Urb1M zQ@c`=cF;x^;d`fZ^^`pNfnAFttGVI2I1F`Df^CtN4Szm1*t(IMoSN1CVBQF5ls`jPr-mv zDv_}cCF%M9bQCZOpWLv!AZA6ioc*|nDgRH18BWYs9|a7Ut1osxbo(b__9fyrW0q?m zX2t&rF>^ip0iG_DP`L9bp!4jAY?#@PLhT0pZIhw3Cmsn9#1Faxb!Exf_nS1L2JU;tODP zaIogxoyAynC0Uq}2Q`ul;|0m2B*~^56G}bkC2^=oPbf*GGxVS>CS7*>LD}es$}U3L zRF|@H1xDFxkiHU-a0Pclx%0T?fc9p0Y-K&D=$}L+xl|<2U#(?fD+;fOSA;wQME>nC?X; z>1yK^AKfHv#VX6%vC68h*lI$b>>Uv9g6CYIoglI4bJ8>|q(#U%uJVv5#`dBMSr#~T zlz#@uBDJ|<;{5j~#3Xz!8`GIA$-#%i5=v5456jd1|FiSkZnBD~y*^DFFrO64xp;oJs< zzho;()E4$)Dc(IBDQR7Tm3Ml!(N6Yap`NRw(*4wXjkQa?SYrefKeb`37S@|3T34r( z;57khScVqFK;ORt@Aqb)4oy{U(@CSwkvDv{tL4DC4weJw#pBJYVCCE~c&c}4;94b1 zROlEf!Q9U~X7>jw)Frvneh~&}RTTytI~mAZ)*OTTC?g{muWtBa5@2%Mn>5U?szWmX z^=!Zlc+xY_)Z?D&;yI2PpnWQYiL=ikjeLV%Gy}(bR3ofNFTRKs`yBRW!ePezsh=SK z5lohjv-jDa9=p?VQa#(#6Zc-YyX{W5;&6Vpr#m8g;LdUHiMs*!$9I=3K?XXUzr=kv z`SzJUm@M>m*g2-&>I!tQZ2sw6-@}eyg}v4HnA}^nufpCcmU@eUs}pfV_AH)D-%CDV zInGrD;c{O~wN1JIkvE6Z|OCGeHXs!swFX(~xcl!9-t&aX}>0>gRZWvwPfozio&J64@TUDeyO zURR^wkK2E9+U(~%)S@`Wb_=__`xh!#$G}=?ic=NNbI?2MEBhl6R&eN-nBp0{Hd%28 z-St$am%(>qn7Q=@bS;}FI9xNDE~W#MkC6AgY^ z|Gun+`NQ*7X|o~DM|sFaARlSg80DQ0gS0h$S*$e{L-_|pJ`?E;A?g!w0O28xraY|C zNF|SVX%_`tVhMtMpVElB?IuM2)xclJV z4RK0%`jwW{%>Njp|x4}sLcfj5iWrQ&BMK> z4sQtJ+8@ClFlKe~g(~e5yfeGOQ{oHhO`ub@d0<}~JEKuh$WR-3gg66lxY-r$Ky}!m zI*fpEpm6Hy8{n=Q9+f`O@EOo0Ag)2Skrq^Fw815>(b7e?%@$31RNbnR&38ueh_;U#1bMr8`_T0trB>~gDDMj=3UF2 zy^XR%jZn83-g_7Va-`ZG!x8+FCH3|FZ3ufACJx|1ac@jqSqv0K6IO{W?$ynrd=b|5 zf0@Axi-bOj_HEcNL4`lga+x25<@_Lx=I2!s1*iVMjujZ`F(7_DM(O-YyPE*@^xi$^Hpq+gHTlhxrg&^Q|U62nd~%jdY$$bDn4 z(muiS2?|?L;HXQ1au8621p5wvmSY4Ua2X?+K>Cj9di+FRi~ zzV49~dMEp!$Q|vSfh;$4E^4Xm7iMQ_ebZPlpWH?M!0brMRm%^|4naKqz-%OTp6CZ= zS%2+UD%Puu{*lYBUFaj1%{o!>SRIB*kE`a8cCt8A%klOn{l4rFt?wWf?A1a>*vLS4Mwtpxa&$?(0h9Oebh7DseEKgf743n8vq@AfWB-QmrEyP)ekKv=FR61CT zG;(*XMGSw!?P@3R{Oa3=KEA)gJR^m<0@*qgY4OjpSW_|Lh#8%qyU=oy~QJhm<-HqdY z`=?kBEhC-9h2ZB-gAg4LNleBMaMF+25+eJV)@&quG{AJaN;~R}oi)Nz=PRna&?b&# zofAvwJ5QE+>ZK$2_6&;RT+V7`b0hpJ8eV8C@mF&o&^M4qD@-QNe86~DGrv>vk7}hO zS)kY1H8`V%Vwu{#k*pQ_Q)@Yj1-rjd#B6C=-%)HZE7o2g#Tqq`&49w~sH#~F3Wsaw zN3rJhtc8m0Hz$Q16jjrrMzdC(P!cbE)l+j(vIqwl9K;nniCg9%PM*qz+!sHCuXJ(J zCV>TGHFY#=-e($!^gyHK$V%Dr9R0}iAXGi5zyfmh;^u;@c#oHg?ngC%@ntnVKpA~U z@eh=LFzQXUZt0lyLe2gZYZM`=ME}|%3xU0ufz{3#xX@0E8_Ajn85H%s-yWWY*frW8 z>8w|(>|tG$r75Vbmw`yDIR$4h+c(f1n${v%rZlb0F&|5`oH49b0}m)1LeVH2^>m)> zovGj>6rHJ^8pGl}ilgwmfnRIxV_Cdgu~Ou(wHwREu+^*W^Ls(q*-T$W84$6;3rnzI0Ndqm0L(I0aJEqL#J1S%* z@He#1U+}g(YKfNBn8)%LmeQwF%v$8L%;??%1%J_!XEKw(C$3G?sa}`0qkUJ$+L58v zn~ZrInJDLNh<5l{Xc5>0&q95s=xH4jwKF|42fJK<-P&A!=uO!$fW4Wb8BqCu&C~vn2k?Vy#~>k4UV@(?9Md1^M|{AOhK`^pdnRE0_6^lvT47oasG&;!@1uJx=_OGcJ)t9utajUoJQSdRPf`3a; z^LeeoG}el{FV<40v1sF%JRDp*-cZ&2RTfg0GJb-LWY8YfwoGGz?4Y(E`MvIaz-(_G zWVT(}RRlx35Y1zBn!SkT7(&v53N)?LbQazqp(c5`^i2+(ydB!?=^!sgC$Ay4|4NYB z5i(nXO6F@HPX{l&qE$|ZHFHaAFoQKW4?@Edp+|-@P~voc`N@OYs2MDwsQ_UqNKeL9 zO5a9^3{sU6MV^9bIcv|nc6Q*WQQBkpACexRnhF#_?K$pRZhpbmYpfCcR^ zJl;oUlIHQm-5omYXy;r{kOgnw6Rxn;k!1i#QGhKFf5@gwLaNCq3e(=3$$GJY8k@yB zXfMoU^?B3>+VELyJ8Pw}*({tax!EvQTFL6FDR{J?bUplC!F1e#i z1dlzZ9h(gUrJf^8k@o9s*3iE$f(o=a`6&iGYmsSyi65E6{OjIoi_K9QJ`7ihmh6F- zG)B&0o>9Bs*O#;h!{X@kpU6+cvQB$(4r}83vm>|+v?LzT4$fh5bx$Ksfsj#apJ`X; zunvA7I9tIap|n=@tijv}S*F))#WDE19MuljWGgOz-JPPyc_U zeR*7!MfCSGGxx#;L_l@~eq^fz7b3%7r*&%D1qVwEYWnfTs*jk=_z85Y%js z?dLpp*$i8KZX~#$rF8OcJ4AfTUduC(Y)4vEG}7Eu+TmRwT$hZ)s`aWSsOwvLP^P)s z(=@9}t3aA6lW9d=cU5F|)REb5;2bq?McP0MjE>+BT%Ma5S9W z3^6%i3`FCvIi~|5_|vQOBBUz>awYPzG2hack(Oz4LNo`7<}cn(atTk3pg_RF65*BI+tnairzS zG^5-Eq)E|O^)y0GC8o_$uH2Z4(YxR&f6QdHom|YmG=;+Gjf)>xtYE z7S5*h=?WI)mv4>0m|;!6gZLLlP88)_cl8ehE6cZ5MXAg;ka z^EG5jybef>FFi`(brosqyUM`xciWWbu@m|{0{=w90+UW7dHA12dOi#6G6uR|C2dR3?0{0&H|Xbl#B5AeFnmkTP-h+c-X+S?S*!`_<|WG4 zng2u8$#A@NEhz_IGDoZBcf*q3wtJP-?6j1~dbzs~kk?A)&#qJO3oO=U^G<|Sd4sy> z2$kltg{Hp1TGk=rT@R7(sp19Z*Wv%OX}3LF~9K@maY7YIBur!~n1&~JOP(}g5$@U#%=)&^&KS*9?MX)W4!10PdJnwN_p~k-| zUY|;N1#`3{t+l@7Q`+Kw(SXUI5~uMWm%J`pGCTK$!bz>-bdcOrMx@LTTBYZ>kweo z!XA~R$Dl14W6hbKI{e?kjrXk&H#k}U2ymr$Lw%eB?+&}-PIGgKI={r?IE$mnFR?Jw zX?WXEX1XbqieF+u4Nl-wF7ZWJmvkw?6yz<}oU}vzflWO?GQKM>v2k{#iFBS=v^(VP?f#2kfWL2o#NU}=|~1D)(*@G@NGQM?Qz`{{b9(qLm3*JFn`PxrsV1{s5jnm^Lzb@n;=-)in7&-JW{ zD;RdA1#4;hCKlukvtMidtr94?@@CfV1oGO%!iLTYM*=V(4_}Nk_8zDLi`uEHTPHCx zSnsOxtElYQ2DhN>RsdQiney%ER8D5C>C>XRj&yqp6?;j(j_EDI&1DC`Xa6g}ilo~B zvP_y^$bvHLv%0o!_7tAk{}pT=9W8iXa9zrqDGzpekF zsOac_%rABQkN&R!!PRdA_)l>iz>j{c1L*o+0fN0d17uY+e*+5&u=CKy=rnwM2P^sk z!Pn8xNi^(CKfV-JZ_I*|No-_oOqCX}sk(}9)~lE?0~KxXKT2WO+TwA)&mckIf_-a9 z5c0%2u=K%g@NNW=jRGzy@JK9-)!-g4rNKZYNxVoE8*#z``cAHGCRtxzubIGJXhsb^ zw$P1^E)_XfdAwc~mPCs;u}B;yB(7&cz6Px(>m%wF;{joGbrVYt$C$Z+ho1v(a@?fs zQrnQz*d^;J^&og$!W{@Tm_8kdUS(Du6iaWt%0k?H&{xT=VMlnCjbIg&x|v0K;o0>% z0c-PCI$DZ#_VQ+C@%gdk?NZiPw0kpalJoiN5Yr)y{fQwvGnAEFfv;5+1pCdS>cp-< zoRj|bH179H?rRq_q`;=S9nnTV^xEClyvbl*h)V+IRXG`w>iKM!5)TYTO^s{G`HotH z>ov)&WDA%?CRW(!*O-6b4S=Rv0weecf|;C9M@!c*@g@JYwY)a8|23Rn0r0o4v0xl; z{rMVeIRHlpZS-sC?!x~HsG^mAe)Mf<--h+0#Q;8v78kSTFp}*kMtI`9MEbHA!upy- zx>3xc8+5_oOWk=se8eg7PZpX3%faBbxAiQbwB3J2C=UpYxGO@Pt32SF(p;2WjrE8r zbY;h?F`w6<*?0rKq9qnfAC|CGs|(Qj80SdRkhdQ)(k|F-4KsnCa$@|4tfkD?vjeb} zq*jdOHk2|-L2kvgu9QVKw3C~RLyUOWbOc$!`(oht53#;HMl0xm9!isx75{fr#ukY6 zWt6oAXSoY#?iSW~auEvMMqYNQ%$d9jH~&AA*KPBpsOt#;`=6*Q@4q%ykGlBq&2;rO z7E?!FBmXO;|3F?mWiu@)MsVmEbpN(H&{qJ4QPS4~A^#J7dHq+6{vYY<%&WK47yF)y zUdNQ5pbGqJf3$@`M)S8Crem9;HmM5+Dr!5Z@^KQ3PDpZ_fpU>sAd8m2!PIoT@hLCX^!zr5#Y3aTlhLG{-4oFLRbbUMQTPx|t|P|G z4D`Fhq|CiL(0bDlJP?T9X&Oe{9Yno>@6iSz{(lF)_1&Ap-tRomULJQD+R=MhJWoDK zEB3HpQ<;Tv%Gt|$^}|cCDf$N7D^hb@m8J*23d_s_X_+zOr5Hr&F{R_mGX(sKG<_oG zR2syKyTL&2WH5&SS7s^dNNF|}#^%u|8yn#MV4D6uk{3pFAe)Uf8(d)jF6>Zu^bBy) zVg4-qri##nb)KV6dtLfA)TS3Z(mU4HJ%GBSDd|ntx?5RnQ72_ypM*4rc^&XBSv5`H zi2vD@nto84*CRUVui^WbHu5+DN8cd%g{X63Dz(hzjj7^I7AnsKikll}0?u8OX98z= z-I;*uvi(eeTP9Phy{t8V>PWreZLWs*0&c%JhIh;+R>1K3O&&Ox2CueAwC~YOJ^jLd zqpY`B6#psOc=;tf%6R$3JqoTG(xFwo{L-)@TJkpk(rG2QHdjo)4KgK|5f|igBz3Xy z)}~F-aQ`WEgRk7zk@Sj%M|$VmE2N>qZ$O30a=2fk0gvAPutWSbYPb*UCL@kw_p$JY z_I0$+Ut44B3ft-eM-#8&7(*RPVC;&g;&pJ$lbg4um*A*F&0$#=qh4XEK3L|VPHkSO zJujj?ZTH~89Y4aP#9G_mY4R$qZ+0>!uiQR)h3(YY2qv)dm#v-X#y%F}{-C3=_SEDZ z_zRcekfs6m_fnX2>SXUhI~ws0RLO3}&zJFUDn1yT0u_iI{bd!O6t8y(|= zc}+$+g~_#!fRTfDeqd6Ro~SDE*G{!WcKdHr%a^7X(!OLpDYq5?5z;~z6yPiBC9JhKe^~uuIyB!~6 zqWV;KseQM@e+r#@mo;z2d}08lHo=4{c#rH=?(oytsjV~qWr_e_y*rUK9^-I>gtrv1-4o8kWB#_}) zx-Zm}$CZ`~-(Mxj*fiEa>5GH z6Ak?My_dl}aE0P^ICD7C-ld6O;bi$BE&7TzH#Y$CZ_|#ifc$!5UjzA7X#P#`a0l{h zDd}q7ahEprmzY$tpt;kU$h7v1`r1zO*J`rqi1Q`G^K@(r6E z?m#WpjxKznYSC@>GJX6Fw5V2v^z%2Ynaj)~ydYdijlRVJp^bKYi%A_#S>Lk2@ZvD1 zb9nn{@kXfDQUMrPq0_v zFbMpCK$~smYir%KOP|Fset10sOPqoGM;or~lCy9X`_W&pmCEo5vbXXiZ7AZ=?Uexx zKw-FKrs6>O=V>K!Z-o1Z3TGnS0^DHNXmUhjB~NCTdr=_uJv{I72Dj48AJ`X- zHXx;Igmm_o1n*lKxSkrW#cMhSk31g}WY4^|v z)$CE9Fn9sT1HKlR1`XTkD9Sno&xoJk2(o)R^ZP=5y2rzF={J0OK1NTSVj*GRyZP2> zc%TKCEvz!v3KLQgSV6l_u_nP!0K#fikx`9r%64)J-SHQL zKj-uXsMj57+4!SNKe0Ed>L+$G)F3Ap{jt#oSO*b_LT7$qSgWs4_!-0=I8Gm(fs^*zlynxZ z!MoS7g0a;9ELdzadgm;&1%v>C^sSM1pAR%LVhj4eEMHJ-=`tLp-CkCYr}1xY&~NA1 zK-P{j&NJWOQ)}z0EkQNO2;x!vBuzQb9_5q1qVLbMwv9%9CEYC%D*{>DQRVfu>gh4+ z^ed}mGs*J;OE%kp!Nb(!0(;x#S^R9PinOw|`3-9|`t(cob~rY+FQo@0*OTRc_!`02 zTD2QX(=W0uOq2h7ZHbpqkL{-3m)UmnVzk+wZe3=7x%I)%>uaR1>#g6}&|&vv!NUQ! zhuVO!B&DjWlGB^%yP~?3JO_CKYKMxQ^~Xl7zrmV~sO%cyp<7V&s`O7%#$`9% zjor2Dlg)aNQNJnqL(b25B zGKLT9^)3% zW{d_^7V=Qq#CREhjO5MmNZTCF$FtbVK1E(i^emO4XHlbC(X(iT6JuVKUJ^En+)dot zwJbKz=U<9Mp42MmcZJrhuHbCu-ieaZiW(s>*!gsOVX%zN++Vj`X^^2S7-_QfD)7pOfjM(BNROY3SQo z?2u{l@mvTEs`dmAm45*F?U0}555G}IUO6t1p$i;K*G{!KfS4S{RXN zcjXA_OqWrDwv{uTDvP03^8$DmZ51xO8*_01jqNT_x-O+!`19uDtMK9?9kK8bze+y6 zyt6O`-D?evoC+pW$!C?@Zdka5-7$*--T5s(e1z0?16-G&d>>|!DsOPMYQ2-iKKmYa zYH9E+x1>mFVVxOhnwimdaufR+1(D)Q|i4^6V3CyGp)9FGiVh`uR*JxO6gVf z8i(7Qc6yBmgK=oxs)euYEocpEb-6JZ-!Uj>U?-W7;Y`>@qdpC|CkTC?qpJ<1f*hPv z0;<0&W>+l#D`t|CB!}ER>~-uM-C2CB^%Tx z)}k(n_Y-=@lZQ60@UG)i6~486Dhhbo`uUp9FxyElp3Xc6Ji)L@$U7^3rJqua4{z&N zR*!VcB zv;zbz(9hP?zzyDTPJ?2XucbHAw=AtQm@mb|Ed#`5%u6|^#mevz;}N_8Whb*nhV9qp zS#thIrmsM`qZtkL^ENoB*$fA@rAW)1TTV>^c}8f$3#{P#e8cE?m-J9De}d}>_l}!ubS0Q~vOI~x zErH_dfe-{A4B;=bzH~W+&vuJP#>waLnhi~F%x9Pz;`1Fk+n9GVDeMzNc?({#OueLP zn-|IpnET4Z#)XqBW_62&62tiye8^#PX~HMD#G;LFR;WuOwlA7+KW07x;PdE01Rr7E zjh|y}{jD4krTkXd1vuCjaGAG2o(Oxbpr?l6T}|@}ML+q4%aCohBr5F z14bihN(^s{!-M>I-W*;iz^Dx{QW9u|r|sJqK8o=*GpR!?f6M1-!0uWYfE(FDmp2&p zlR1u$oe&goJ67b4X89fKNKG%UVe2U4G{>%fQcd>G?B=1^pU z(E+$I1RyTBI3_#Lhb|=Wu`P`bRN9BNa-mZPE-b0-KqZEio=oInJZcHOl*q#w?i(ip zqZ#Cr1dR3qn+Yh6%MnSu3$C=~BGs=<4r%9SfGE0v7v(O=Nq##Cle`n<)g*VrB;)VK z>n!Iwo~KB_(50lK{1fMkUKJu@xlq zTo$%?A}ffysTG{SM&JfD_4nvPTfV->`3YDBH~tMa0%1BXEP^(nY3dOX%axjxLNMPmNQ?N> zRXwFwJVs01jIyYWYt9^xtWFSh%&B8gl9I+jEdHP~k9A#&S{lh9oMV4-_nQaLI!^{bn8r3@Ey8nC|WSb95DJd3DimX z$8cF}MORqaw$nRZ`A}0RF6C0yV5r)0-8l9j`mP&q&c>6uJD0a36T9>FVY|k`3*Z_| zsFGPbg?oj?YVaOf)SV~zp6pj=PgsfRcGwgA2lbE)>UMiVq3v>aK8Uln)T1XKWVV7x zD{bp~a=|jH`y%of>=F}Pe3#1L*9z8wKz%hVoL6v-q$&!Nm5qG`i)d(fJLoZWrix8e z-HUf%`L@vB{9DHIU%S?abM_fs@5gP@Rii&2#GWVfz4&^N-no~*23>yKeSAOvXMr*b zQl|ktl>KcRHh_Du==eMaVshM28-2prnlWi8^NStA{QA)1`*{Q}44^Id^VTse=(gKv z6R~y1WZWSwk4mSqa2oJ1^QJZfc^j~oih(?emdW2k_hEc`(vYD%x~^he{fbo)N_M_e z<*QUNl(!#L9wc|4{DrtAqME9*Dp}uQ6Xg){mi06km6W!nHj@R^taAeG^&<5jK|e~e zM%fsYRi?C+%gQX_s^qSJmy`l-a8GGzeamf69xK3g8EHKlh9y*Kn>UO%W#Cao!}&K2 z^SkEaIs+D_v>mjf#Upr6voC5KpyMNW8u!UDC@gLy9}$lGIr7v)xw=T+7wAGC-Wsd>Pgx&nSamr)2c`K(D+a0sg11E4MyJ>mYzBZbk~PPDfxa9 z`r0R}d98dujDn}|CdqGflbUngaXCyuWfm*Lu~?T9{I0a@2+@_gf{P7mM5>$8Hz{b%W{{?%jZ`6z!ssL~1&R2z;Y2iyY1hJ@?;T_|cA@6ZxglE&b> zs}`qAGhmM=Acz8M?T|ekmu8p?>?TiOf81#fwiQm}O>uDZ5Gz=O4XZp-@^wwO0+A^E z1r4X~XYg>hr(g({R6mc*GkKGgnWJ$%8vWghn2Zk5x7;37;4if3%%u0+%lu~d#}NSy zo5^=Je&2x*F8?@g9xDV3TT=5`JUoZR-NwCp1NoSYJEYa)@UVjYy5EIu$?I``+Ef`^)PY9N zmV7!^?}qMVPl(kY#vC1t=&1LUX0`I$?=1$WQQO(z)3p`j>Q^l6sQC1qD*r=8vkgA2 zh!p-Cs;Rbc)e+nlI z?`hg@$gkJ2Dk>w`d-O*xkL0%wP+%T!+ZdbISxiw0c<|XM`CswgN-1hmL^O`Na`Vt( zaNs;1I^d%Ym`{d%6DCYddFxKhuNQz?#?_@qLII3(q+6vkM_u`5NSCu>tQdEFGyR#z zSBg!;8C{HW4W?h8;U{qsEpGvD2MKZS0zS6EoQ`@1u1BB3t3g93W+9g+6Pi1XUx)!u zrlN&BvI_`KTI^&q99y`h^q*OHJPpe%Mx}Fw?kOBO{E2iWn4_O$KPUZ=*dpG{bjfnT zOtFjjEw+YYp5aYBZg$c)fDO8LgmCWtG@pZbA;+HKe~E`5$DwwVYdn=Lz_8py>^a^q z$-A?`dUL%Li|q|1KoQ?UITstyT|V|Y#Ao!N79$^Q*M>kbZ`9wEK&JpW+=V59MktM&04`9YFO5_O2%+7 z?sz_Et6srJFxHPc=kpvMIF#Pa=aC@|hq8jv*n6aGd4DBmRSqa z)IIs{+QGNvFW0uN*!^nru!5gPizg_0C7+Gs#C0qA2-6x!KXh{?f5hionCzuYDq}>- z-Euh1E#L{PGnEwZ{m_UeuHrK2$GTNK5NE2pSMgYDI~>Ey@puwdce$tUUC#q2N8pj7 z-EsgRp2iSc%{wwYskWNm$F|b))jT<@`Y{+G4rHWS@E*73GD4(m#~3zmsJk{%q)PFzdxW7;;%>I zz|1&;Su{;{&9C%Zz?DuD5Ir#EK6(z#eUUf!c@kAKBkLmIjHDee@<5;I_T<6%FKZv8 zuV3UH-4Mn?mCm5BH9Xs`oBgwy{On1UYq*KOFrB{N!Q)M3sUpDevuN=W{}N=D3ComS zDxXB2YcZhE+}83ybJBFQL8I34$OufLVc5)Q4G!SsT469xf9YBgi9JA#8rX+aLuG<%sr`0xIpVMoe&)bPA` zF&0ZD{Y{8O>$vMZWqqtc`Z*8nItc%myu5WjRx42(#U_)nQIwgDjiNA6kB0w)+V%HI zrxN1?X(8&Yr2gx8W4CGDnO_pb>j5-<9skTlby%m~ukiO+cU(N|1{GccUqzJ2 z0q~z3SI8spDYI%WvaPc0N|S6W|6pNIt)Gk3>E1^h&$juO;96!aH!D0iwt+|JNFo0e zqV~EX{!9CZ(b?y~J}g-#-BPjt8g`z`d5AhD)3n8KYr#M{BFiiW2e|cQMX}7IeH(ZJ zVjyHRHlzn=!csQY^hM;ekw@{~Q>4fcNe^y>k{(aXH{uX>#84$NaQN8$wvjjUc?E4d zPf!j;Z{nrU-Kr!g;~wQ_SXldO{)lF<*TsJ+0q4KEy6P`U+&|e2tI-DXe~l0FZ;VbAfJ#nq zb~y`HniE(aYi#phgHfa1=R=IV-S#|=Rr=>Y>klcOm1RMwg!FAyc;1P7W^6m<#kLvg zZnVFcKj@O*RSUyC$gmO~XnAXjG0N1bgm*-^+UzFAs(85ss{+9$nxMaiuDYD^N_lel za8!_+-)waaAfR-$@?G#P9gKAGi1b3TM&w(yaJ{OVrjKp)uXO^IHe9U}50nCQtdn_@ zrT@_oTDFDv3T|YFFRQ<5hwX{9eRN?9@9XvqIxd}%UZaey{4>}@(9(DDSPl^ukmSs5 zAj#D<{dMk3i5Aw2B3?(rWSOvqrBm2;{wZ(p2>rI5r}~t|BtbL)pgOJ$Km$a2Mz&83*4Xlk;9r~9dLOOV$m!%Odr zoXNi2I1J@dqax(E|JkT3 zKDy>c)QA ziT!*q*Cx=^-8?!hKc>P{^*l|!!G=d|7X$s6S2M1|P?t|-yRk9n)0y2oHYZ<;@YEU5 zy#iJ-biCBm5hzoZ&gh@&_D@Zl39!4Kxlb!eht{r@d+ObQes-=^?~eZqwMeB3X34NL z_A!v-tU(^A1_jCM$vX#2n7*Zs>r8T;hI7O9EPIj*f)=2nG9K98+QT1aK59^5Wy(MU z)(9)2`z5^zv(`lvdu0OswuN^My9hl}ZcEP$GRA!|Ot%g_AeBCNlRpE4OTWFmjsFeA zUUO*6mhspyp?N<>tM-E1O{ZaR@q2v`g9dh65Xl6Kok$Az@d%Gx3>t$ez?$BIete7f z@`vc`s@v&8w}4tCMNs--ZeLSS$1$08v&2ylea}$*Bku>Dj;)j=g3?c@8F&trqfGzpwyxyy(9-H`2i(G&p_$DclbCr2TDbh^e%7gInxOvjMlq+si|iI z7Gmc6JV=t8?=x^IJA-*1o0xtbbWxd|*^I{T=P^;3yGkpJA99Izin*ObU=SdbHaDCU zje>}q0wyu|VS`D;m4iu~11TyA91xb;7r{?RE1>1&;0g-*$1Q-&r7u{JEw=(zX(`9X z(0Vv8d%_W^rP>kJ!zOO*Wv2J|fcVyc;Ovnhbpfk1*zEJeiIt|0!9MmutduzLehB5g z2jlc5+o<<(TvG=e%Bmgs*HrNV|2Vc%%WbJIgQs2%cIZez>k8=r%LC@0mU?xRak8pm zX1&MJo`bxl<-Uhm!9OUEXG(aa&HY1AeS`Br^d+R7K^W!L*LV<(>;VUgmk)7E()Y+h zSi^$)8myMe{6gC^zmE8G)?g3Q@k6{l*MsT4le}327%d{ucpkRja?1D!vS3rvKjLc} z%mdt5z-=i}`Q#TKWCFS6Q>7~lrD-2SlIoB}?q_&Qr{r=Z!yE5|kNJ34MJ$=glzf;k zHU|OLhg5Nxj}N~kdjTe57=0|I6V(+uz8Qxy?ks>te!?r+BNTsxcQkK9I*u8S z@VCwL@iR_B@ul>me6V#)Jy1v0Z92cwkx2K*W(5^g{R_&D^uth99_4;KJX4(tljT$X z32!u(PJGJm0V@crA!JyD`QI-*wQp3zM?Cc1JsaUz9y#<>7 zCc-^7#MT=+hNgbT+qyl3x}IZTVW;ZP(RBG|e0C^2^o86=;8B&R?7LD_17(+4s}gC@ z=X`)u30AK}dJ^sb9NxNv?d|3~&4RtM>D1?Zn^Ubjz}@o2e}U`y4_t)h)+FKXHE`~C zoF|$lf)!Ejah{W-G^Dd2Q>oZvah6$%5HJH2gD1fZ^YQ}i*clvP89ho;i+Ufx%5au> z_dmL0m7H@RQwU?rSNz_VT7cY^%CgJj z3CrSg1R6&8a!U4YW`@iNv zjmoU)i3)*ZFqFy}8bxQn=FOaXThZ$F-f9UH`wdUx5t;PRH#{UD7-&f)O!7`BY9vp7 z{W59AH#{Jup;O+e8}ek)zjPQ#%#um(e#7rK;h?uQ`F_h=z+TYlTOQ9xRnX*b`4qnh zV1t!isTD@*+o9!2%xKzYJjh4##JAL1#?y2F^#=K$G}w&SN4zQYDDcM`cEz`B(+VrUwLFRXR`EwnxU`r| zZ&&ff49>UJ{EwC~C?!{D;R1ea2?A83v|m%Fv_(jhGuGrG8gq(IZ8`|O;>fQN?AVq2 zaF+X|D7p2Xe2mo)QZhj`5>-F4{Ky-4jKwjj1A!(#@_gBkA9*wfK41RGCm8s2JI&|d z%3nom+JBk{wFN;X!}CGYeLw|{o$JWoMv(j`oYKBSn$nYj58R4>!byB3b^nR?3)|WU zyrohrcpT&Jd=Q@rt^N_(_7e=F*>vnD-Za!8*8FTliUYytFGSO)Fg=ps7n0A>_g~93X}iAtR4bSX=rs0c{#^J2Xm5`_MZSjk9K;v7c=0sUVE^Id z{tIs$jb|UQwDPml?JZ>*Eitsz9^mCvnTJ!KUvMKX4o!Xw7vl;IuU5-O2KZq>^M3(f z{%x4Smk<2{o?EH581tM@HNWHJc1E7i87X6tQbEJc@X>5I?LEW$Ha&% zq6Ew7jsMajupfC-(Hvd+0MV^Go#kEzxj{ZT%VRM!8BF>s1^Ja(bKxq}P1C;vJ-3jj zM-@r7x6ks1Oit+GbG*Iu^Nl9g^L%=vuTe&}n8R_3jaH&Z(f1d4OWJiF=9o%4eV%9H zytT)#U{8l=%CCGt%Q=mW5z0(4=AtU4mic^UG$@%5lp=%qi~t*jPGc}1R}^q?mczZU z`5n~?W@G4_pCC)W3O5Xx2VjjJSZ3!eh;4-a*(1J1DbA8_$!KeE7O2B*6=yjQi;J9~ z*_3yI$HJzz?E()Bz6WhO*hY>zknu0oF^=#>x&R|rCH4A^7v~HHHoK1~Y|5Om$^9Eo zd)l%2tPUHr?!d-W(gM8EQjHKaPMe7IWRaVQ6?xW(_G&(60>3!9_5siF1lfd0^#mC` zdXc-gsSh*be}I|WPQiSLwq4{M+U9GRmMFB9o!LfjivJ6%+Ulo319FMTQJ3SQMgeIY zqu>KPAShgdeXW#IF7a6NMU;Al#$Dn=Kv(-N@pM^r7>yKf#%xeRM&J}tY< zTe!vm?HFq+y?vPn<3`o-%RCZU*DmwKm=MfPL`}`^2&t4gdRk@gDej1FaMg~;QH3JWrH{H@2bLfqH5%FV%Qu99%G8!#EnMRN4u-4%%z^15KnuZDcm;~Gy0ehC#6cfth^>|u&4{e=EON4dtAnk(>g8iifwT}=5i zHgFn!ollS^B)sq>g-tDQ;95Fp3&Vq`ieGRDokHD! zIs=+i%PbuhMAY91%aHU6aQ0`#GJ-(^{;XED$_s%eT66YiO+||m&093F2DTWZF&D#? z)qsv0GTk(NIK5MwU*tj;Yxr{Z*Q)s<&~gf$)zP-nHCOEtex`I_yy_E?Xh5rWRV*RU z;B6LzC7?V+h(a!2~Scq2X2LodR@dkg83($#9$JoIPew_N#!?)n1! zmK=Gd2EQ^qe1J&z!FgwZTj1|R2G*A>L>6=U4A5Qf0;78@-V~;fdg@%WZ7uqj`dEV zULfjir=kWT$gdW8H-&sFAis%@ASb9^PL$o6_9ESCAYOBM1@qB=kl_qf?kSoB*E61? zsh3(n)p*-V&U7!$|RgRxlK^=}c$2h_dqQeTMY%0uV>-N=uf9UIJ*4eM^_UM4HPD;8;4)K&qd&IPLZ}5+^qq&FIg(gumh|KWZZPw__sxXF*^`HHbFUPylTegn=;Ur~;?VBYZ) z?aUto)LgpmC$@OMf}j0Ss{-O|FZ9X~XAk)cKSZ$m&Rkt*gQgmM6vl9lvqR`1)-@A0sC=cA=qw$f<>me1m6$R z{9rLYYz2Ngn^67^FnGXoNN;CDmFHPl6n=k(fl?a~&d;8=mkC4qC$z1x_`!S{jjbd- z6c9hcPiKfJ_R{Ym{V`d@m&)>liQ>YQT{E_!u>?BaiQJsK3Wkj+PVxuW%GUVZc#X$2^U*=gh5{-sMOI8r=@v_hFC-S>P;C)w6S zih&&RJFb;oY$`h8SCtnfM2i98P_VuABfeU}7?2NS1zfL_PoWtVOK4rRcm<_18&X;` zF%lx)%gw|}B$PL#k`U&$ zHFP5E3;7z9DIEeQjzRp4??(Dj$iC2pjPG23{bPK4;~T>aaV0?i6XoS|Yesp0eK+!@ z#X;q3f%*mHON)b1-d8^>1zL?xbO4?==yC^ey2}*Y5u3ni>faIDf2-aY8Wf`RlUVF4 zI*JOg!U3H`tRD_Ud~|nY$F8An*oy88T zCO5o@Mtbf&27PdRcg>VOsfPc{1*+(R?d=O%o+e`Pq|M$m%ugfqq5>79J|99?;fwF; zi+VGI`E5@ZCUFa&ByL82nT<&48y~J)<+xax7 zn;0AxmW=6tQ!97?Q?(lghhn?ILAot`M`ycErLO(pCRFWBO{ z3k&lBGeUBugBf}CpnH0V$5Kof6G=$*g}`-aS#2%OBa#`YdWLhy9uRZpqa$$?e~)X3H)q~{ON30Kl4ZqeD zbjj{UG*pgbcRJWdOm+J;(ZC9k%=?P>Blaa4GnE;vTft9~yV*@+Xhkaz)?1LZAGWy5 zG_0TKY5t)*u2;~j{lpNA?#+H8n2&5tf&E1iTVm_oUwCm>j78N!3L+~*sz4i=5g#gI3OC}Xf#Y5ok=0dcU{Y2JjNgGhfsJjoW4=MWIVWAwlf zu=`=OcZhhBcjzj0sQ~IXRBUqbK;gI2sntDj*=X>fSSa0w9u)1|jsx^UNEW!;^Pt#* zZ2|CnDSwzKWKUAkaFJ%7j1nO-7n{^290ykp7Xy6Kkm&4I)rOKr2wUKpPIiuavu4*- zEH+FZj*LR6W`u}t@o7h6E|f;2c8P$}p~^hsTlyDJ*bMi^_4qUHR!6i|~ft=u0);5EQErl%BFuCkN&lLvf9L-6S*G|crcE5IABc(qu2Ck95dy0_ z8^#FGA1sp{bcaP?1)a?j-FR>uwHhN@izZ#z${cThmx&nEpf2>(7=Z|CT6#FG9V3>T zJ0ta1iq97Fg7QNosIrCS;rfqJ#=U^2F-CoGDy_<~G%O)x*hb^*6d7v?7= zw-%UVLvm{{7LwbiWF0G(vXiuJEF_#07;;EsV?mYsD1IFNZ6tl1c-v(rGNL-uz=y>f z=6mq_PjVYCy23WrZ@hReQFI;X~nH8q$Hjc@&hHzk7_8 zZagX`;5vq8rSa25Tki?&Bv*x_i&eQ?rqH%&qOsd?AiEI#0B4ydo`6}#3EErq^`j!( z0QTf`DgOYN;_Cr4WtvE=5Arz0KL(Jq0J14SMiDUdrLm8RHX^GdZp^uw`->r|_RvU6 zDQ_lZi_X-4hPaQ#&^kCbvxkgCZ<&}#@AOrWVd{IMD$x5RS8Q>47N2d~*J@Lt)cpzZrh6}>{?!iG zOWM_{RDsm~NwLrzgd7|CiuPOVMf1!$^Jt$#d~Hhi)wsZ<~7KClxLN3H`cU zSoo4Q)TKb~wJXF)^R3pB!4)kMEy(|Q;e}{y(a)oGpEmT(^Wara*<$jAVD9?>Vls+O zYAx#q+WPBaE#tS6sLPk48F{aS+V%rwtrWien$neGxM5IlrgSfK0p}dIFy01Ln5G2r4|vX?o2qG0e`18Bsxi} z?kPAVI%L4{^b?7B^;5g!;R#Xh3%fANR(7@9g}otp(X_qhP|92*R)UjWStFizRi~aV zan$g8%vIq_;!qTL&?RSsk>pR;u-_GeKVfyjlo6%gVUjM`)W(y6Zr6&bz;D9K zBDHxzC!-TuQ@sKLVjD+806Z6f<$fm-Huk%dFJr%3OP6033`1S<)27t-fzvEn{3;k;UpR}3zUJrPE!mFPW-$i0631?apc@td7_--F1wj}p zXBnUnx35o-xft6`Jq+%n9>wB5|2MGhmtQnmI|EJWNwl_&5LLF7WBX0V_KPd!YWuAz z7X6UkuSBS0>=`8@J-$pUPpv)dv81P`PQVoq>0~P%Z*zdxEr7I>3}lTG(Skn^V;qOX zm5Q-AjMhsx*mf*J^C`@*PHW0k4-S=y{Ft`_p~$XNbX4X{N9mx4Cz>1d@D-4R zY_tQDhn-2D;aBbupxcH4AKJHxAUe5AG=W$RPpX<-;!oIg{@e}O@HtA|BkpT34z>JH z%NKt%V4#SgU3;+Wm(dq{paLlfaO1VGX14t{@ih;dw5U4Ahh zO`UM9X19!4bH?`0d!mZv{MuPQ9|$McX#IWsUzpomKLtH13mo?)4CDqBy%U1xo3etn zcgZ`AyaMD&rlB{&JqXeY0wI*%Ca!-~Gx&a><|T25{2O=(ja#alQ|v(z99Dr3YXmL< z0+ZcdMu}?*Of_-Wm#e4!wiJX|1Xx1lUhWQA7BJj&6(DZi7#>a=R zL5ibYsYjaa?XE<6*C=ZElxSr$9Tk}@Ok(^U=Eq@hf9b4&VGE>xY0LW*lnl?1?0m&gi!J8UKn-L61-7{3__>gdd;xY*BAfH2 zWyeJuMz)IM5RDnEO<#)H{vSu!`_h%`&;C+FGU(a|^wF1Msps!jh{4D%#NT?#`bzwR z?JDzY5y2o>=Y1m%$8>ArWa~yq*))7h>!6w#xVv5 ziqn>p;uYHdBM!uC%g?O0mrlruOkIa4^a)ZM1 zfQNHCLXZ>xKr9bv%F+dd=`+IdhM~;FaLUXam01<5GAAHY`ka)bfTqRkJ@73_U46s% zuP45nI=&0z^v?K>aD2nhw-dgl$Bw-LwL=s)@1P>q?62vF?YHqY)gB(AD#TfJzi`%1 zZ8NHYLU>jJfHY5og;Z*ngd}BLhe}kI$OPm`PILuq!ayiV4^s$4e4!f5h*drEG+Hov z1bwjzdcpHlRD}gSpU%p!sT5reN+`ovkEX0@u>rfZ>6CaU$5=eILf+Oerp^k1jS>KX zO>N$*$demnZY0+SdO+9!?+k>fC55o{UAv|(-q1C*SgGg7sIn9mUWFmkQZUC-A zeaQmcQ76K_9Z>p@qJ=pgXq}`fKZ=j>9zl7$KHAp#w20(6F#jazI%Yb|F+gwH2PmSI zUI9gxkcSg%W3~zt^k?ubk7SI&P0*jlw>*-$eQ;Z;!F>*S@?`c-d3ngg8Nl~Hi{buf z@!Jtm3|j;FqYh32egP+m#9UmZBc0(JAA%Jl@67}u37Y6H;w9fo&4F#Mfo*6gO+5qQ z7S^USke5Hkc+5o-hic?^R(y`Wr?=I&(}lC5L#r|k-C2*3k|#*U+$ZR3@qH(BH_!v; zM67i@0Lqhc3zUcn4Yljp^4rD!?V?>DO-1K~Z!Mn19;M9x-?x@*@x!))iyLr(Vp z5KTOe^3|yKW0aBXxjrf<4oJ_d`|E4t;M6$U66}r(eS)d+O9+m|{ z>DEN%N^^b_-Y(-%)UPqE_)QFFMRfKzNYn*3pNpaai#K5BJ77NmvDH4y`C7fadqe2A z%K}HUm!NpyqTVHu+~6cIUWz(P@TcUo%1h#M9PrtI4c8&x{2J^8v|ip%bm20rF^LrR zyGVAQ0dSE3WeuU>zl*hUuW|;oK3M9OK{SS&La6;8!rOPK#aJXMm=th9C9012Lll_u zrO)dhe~4;adOvm%Y3$FSSg0L8FWJvoI>0?P{X3{;;wIT@_8u zb5N@}+?~WqJ}|@xzIN%V=w$AM^z)ST7dFLk{5&Fm`nqDSRlD!~B^H~npw}C1_g{mV zN?ypn4$Zz#kcx(ei&o!V7v0P`DA9^gA`q9exS8Ukcwbufw|LNeFLG{?enSKp9T7JG zuqhHhSBW?|gCl9y7)|%ZPUdL!Z4?UL43XdA`|_GSRN7_8@`bOBx+&DTkMn?<1Lfu) zjv(yIZ^5e8lPYduoi0N8Hstw_c;2@V)2QXP(enaeeoz8cvOS8O5~$#Y*-g_|(XoF- zpl^RvyNPn3;8Il&GjQV{k>cw3XjdaT^KwD2-w*r1vKrwRsB)lwzYB3k`d=GOv>{uK z2=f{YSdsx^o*+QuM6gDTVy~^rFeUq*Loe4L`$Y#@S;?!OHLY?}@W2(SDVmv@yDD%Q z?G?toizLQ{aP0)fze4E8+W>y@_Ys^@fJ@|SK^BA9w06caLmAopbB&DE`dlOJ`YhNQ z5MPOz0=-uOh3;rxCcS{)$~(eTjz*-BHeT9in{ZPPBL(iyp&q$-8q3j_=ecQq1BGpb zzdiviQRXE?o{kfyv91cyh!i4WO0}voxhZ-Zl#y(;*vYj=TB6^k%fb}MF_G>jQ%iSM z9H+kF{?x@}iZn(5yF8vD2=YMpvBe979dyn$7*{Y4@Ta3DQ<{Cv)hZ{!{uFLDrS%0f zP?Te5LT2Ctr&Jl0CRIj;g`PXONZL(9ji8=m;GEv^Tg^|USv=6JrF0Aq&e|f4=B1m0 zsN8HC35hSv#Wceu8w1(MpH{n=9>yWmEf-UuOIIXa_LCO8#+2e}`o&#C!glCYulpI2 zT_=lausrCpm;%k;0h?vC&0<<=-ix0TY@OXquFUOm{EkMIrqtivw6%WEUhy>~-v|$r zg?IK>lCRr_?+ur8Txj+%^)!bA0>pR^Q#LpGQ^N+P;Ck^}A%<|o$L!d^nO+6T*ORZ}Nc=rMuENm}R9Qk{h5?C7b@IrD^Bx7lNnMQE`M)aGP=^pdn zP`66xm=Anm278-cb~}vkC$LQB%Ctc8_AwQ(d+AjlQ*(0xval2Qn6{WF;Afx&L+aJJ+__gWD=zaYT$ zAZ{672r!NIe-1^@@PMVo_M(aS{(-UrO|49osZpuMW!PPTrZ43ieW{pd<1(zPk8v$? zSCA`7DMVYnX?TdK75De1B_XCr z%Se>*@Wu*J7g!rKHVwdTf!cluZ)uus_Cz+0pc|X2a>h21B#M2qqY73X;|0HHB_`U_ zb2}<8x+tUkqo)7eQ1L{Zfm)OtOIIE?`QBS;jgawSHT@#8J<%V9l9s0DVe3}tv}meJSzo1R zrs)|_j8*!ZP4(-jqtw2(+I5C9vuRtHX^_ja=zgM?Va*N=H$BQA>a~cJoz3r>+gX>Y z^<2~hlrrrLdwP|TKEsjjD4*U%m7n5B&y6zRAcBmdoffviNsfBF3nMpu5{r1%KT`Bi*E#f>m+hKyv5wEYobn#FRy zL|{XOZLe^n^~?@xe9ArTy@21c8lNh|Z}}}p`d9YyfzG~sQrXYe`M}J!Bf;Z%GAWi1D_K@(yznD1Ae%EZbSNqNW>{b4~pQlbrl+#3sUDID{DSM0JZhd`{yg-to{?a?OmP1AH zI~vwTe#x!Gn%6}WM&w?l>UMH0`LvZipce~oD|c^;L$9t{Pvw@=Uha)yA=q}|AnvsH z*8ApfgSDsFyYiQ?jW>Zewv~Hzo9}}g#O5p4U_%2|_;GC^U6huNB=+<^WokD!1=e(9QR)VX5T@mfoTS<0iwr$1yOegh<#~t5ZJlA8Ldv219P0Tjc!U?fQJtE zw)YeNm9y@o?X>o?ue>xsJ4x9c*Jjej4szf><}NpLDF3%)E3tKpGQd5F&U+kilfP5d!$)fal-vwcn0GtB$g-RDmZZIzpPNq^n)!MEbC! z9B5*d@b7S%w0mO)WTTE5a=ckc`L{xqQ;N){aAKiQf|>sW6?OVY;#N-CbzA-b>eA^} z+XMf(?Nrd|Uuf@axwCSN-%jZC`C*zHjmVN?D7dqnWOjNYJ=_@-fPO*Lp7}TAN~Sm| z;Odw9&LDtMX2idhshk2;o{L+?b^!s5{GI$AD7Hjo5O#9W8(nIc1Q0WxiYAuFaGp-heP*o!M;Cvf%w;?j%;t^jgjQDl2S`oHo zM!bz_sfwxnGs3xw#L1PbUp-iANlr@|U~;+iwWIkKkb% z(@TyL^JsQ2xm9$YO*;)S@a?3o6?l`Ux*x7(RU~z*SeJzNNO{v?rntYC9Mbhc+@85U zt9M1CMKVk=nz~us4#*1t*@0ivc=>yrC?M_^viFuRz?0?*^Dhk& zK*MR8-bX$mNsLDJl{<(l31ZSu?j_$ux6`;T?@^lBPmYJz#PWXf0|hWf?}!6ff5&8W z;S(sGjV=cH91v^4sgc@8sdj#A_(F4X{0^ypJ`{m=rGCZ?cK<0&$MMYN8w@ zir?#7Cdr3HaU0El5c}c2dc%YAT_TL5dp;yb1rGxB+;LWIqhWyT?>Ghb{6OX)o!cgc z(wEa^cfIT(Xy#x>m^V$15!2}TX&50b_4U)_oiZF1K0E_M{7+gm1AYG^)$!0_5}%M^ zfvWd>LXNlK|KXVs62-Tv;%Q`BM%SN~*TTJd`7C(=&^SL!9tDSk{ht9=ZD{#3@<#Ch z^_?w;xi!LwkYA&DM}K&>Tp&aBdZq}`-&`@H=jguA$rB~`)|*+3>f-bdi{+7G3~uoH zJ9R7mj=5M8T-Hw8am2!p#b;io*XGJCY5WWFPMBjFxlNytJt$?aJVmk@8BX4wcAA#X zlc$KE>f(G1#FkCUp_U7-TDV5X-@%>K?VE1a_N1Qu;{Oy#|AzukISa(=MGO9?z==Qq zw}u2~fp2NoLV1?>oZfX2$RSO=NPm>dk^0Ar8a`{;iK85s;*2p=M6ns{f-jL%&@k>4NP5DVtJgJwzg>vA&h6)a^M7O^U z%T4iJefb7?u_)f7CsxV_E!e1t9@O}r9NZnZsG$3tBWT$th0?%Nf}ukh$CXrNTqha| z%7!>15pP~)GN0Jw54-g^ZSuqUQ}jrU9IZdISsrQ;$Llq^941N;r}RIz$|06UyrEZyqdf^v5%8&yFYsBwm+SJ*=#zKI zy(RG-ebYy%nUip*{Gx?h(0CZ?kY0Iw-QYFt5wr(?6-qOIyIB*=f^W65H{x$^Wsu`x zV8#bS=m~mohuld&w@0o*U9{;F%wLb{%RiN0lm4ZucaICDbAS2gsum;u_NwqS15o2t zjr-NCs`_(Y)kV?{psEk_nP19FB(;w*{djPfB!LOW>|?g;?>eEh=9pPo4&s}(>j(|1 zfqc*ie4OLDMxJYt!j9{gzm}s#@t_{^jT~d?>4g1{y_M~%3%zyjXQ6a&?VWY;xB5aH z2dL^BWz5inS#~|-+U?!_(3TBSU+%9{!2_T3VnFw_Cnhbb`@HKZ(e8t(CP#LG;peF zqu@W~r+hafN;*C3kUc1T0ndU56#kU&N*RR`Xk2BE;DZJSJMu8EPD4p9s@b@@`l3ZM z(#x6|H;$K6#09ytoC0Y43Z0FfxgZz%j<^XtzyLmsye>iz>Ph`C%8wQN_TB&1aSiqQ z{YxFONaNcFquyK}}#1(%BH5a+!tDX64eGMwlMLhlB z%WO37id@ouFi@}oGmbEyj_*PEi3_t2)p`4a+(Aa%-H02pYHXyJXXRr z+J|qTF6@p<3xfg;RLbUdUe9@-qw*e=BN?_-AKrlI_9h*^0g+=KWjD&lx;H|(p=I5r8qB06@XNh)tP+eZb*;D75{aB0jm)`z+AiP1 z!3L1;tV>Br7QZ3UqWCTSPVo$Qj8k0{+VgdefAjwdTN;#95u1o)N$Ct34w?OFwxrC# zfjGW*;*oA6>aazLk~{HknD&CDg&rX*y+vgM>K;Rr73D5V#n%Eg`$_CTXB1_CIEcD3 zrA@OLU!xM|o7|7OGHhbXU3l@v$k4M%h8$Oh=_(+cqg|@f#@oM12H&sghN|3yH&^>x zl{S`LUkUWpW0n-$wcTU|zDtY}UOGIix$4|Z)7_N0;y5$qg(fL;c$qL7=&qz;$MU>8 z;HA3Y-OJNDlHCJn$Y#-mCPiDAMXP?mGdr^%vqV{Z4+*sTF-x-L@Rv@Eu2P;S>V1=@ zdnj$ZK0kO1qfdECM;hY=8a!bpuV|7?JUMKcw~}U=b5NkpPfH;*&|C4h^mc*9wRLah zZOP4sRa1bo$RDY{uQElo^F=f_P~YjRd?xuu^JN;2zOoFBMS$ZJE%#Sqa7OW=Kb}=9 zZ>691S6aB4FscGIhaM7%*Px`DWPPkndDJ4Mw9ya6DEEqzZ;GDOLixhovohtm_L>Yn zEv`<}Ep3(Nid5G@4{EP04HX~O*AGx)R8{0S6b8|^_bN+a9s2NaWsezW3)0i?Q?^NF zs5VHyGExZ<%}`db9yA&_!`XP<{Yp?o-zszFdNr{Nmi(4$-VVfHy7UvXwWmt_qd6gflL0gmC(q& zzFJ@0LbJXuRP+U%Fnu@hDZd`74D0%4`o!7FXuRTU?dgjr7?c;hLwpP+5t1vd6+Zf? z0%f#>+wF$WXRD)%b`>eXQbuR}Xp!=)Bvo{wQN>Cx>3nBeTdcIj&Lfv!IpdN{u;V+b zD^}8_<~oHwuUwGwy6VkdP*NoEik>}BNwVO8_~8XgrWC$b->^U#CyMdpyGWUco!ZPr z$^zVeP+n3t`W}NZbxsw(HsS`L?vCH}y)P*wiAVG?nzGoU)*;_`+DNC*>_wMk%knEAS_JmZE8SyY;AL%IltTwY@BfK3$`{f-j+! zp1)R+ai5oePI+Bvp}+!Nt5CWJ?^_Ss!#i{s0=j~JuTa7uMZr8aiN4Kb5j4=DDAu;{ zD1uGtJR0Lr_?^Fb4h+OH{X>Uh5v@V%h0;%O`^#{0>TbH(GhkhOTS-OTRg#|cj?z`c z0GqsC8SD*N%H1>@h5wk>=(F|8SS1N`Sd zB+?_^rUy4DDX@`Wxk2Gi>%nU9@|)(&nr+<9m-KHpU^xT=Gw{ch{H_vXYvHWh4nVG}OIDE6Z5Ir3I9aE?i;*1jCcXBqGFbeH&cCbJ!qdXE z;~1j(f~WQiAF8=+u-*yA*=tuyeoq&$hi9+H|hFVZkA@&=!QMNE(0krlLIv(nPH23S8P z2%F&vF9Rxs?fd)qh0tSoTA=WqHPm)cdfM)cI^a0aD%K%M{_Fb zkDG!_9pr`{l=r?8F8N1L!TaEZ6k7kjGEiz8sW-l_h+?>k)^S>X(Ac<{_wx34Oj?SF zC)+qVIIRwfCf<40Y(*2G6fRSR&4g6JDzx`U{JfT?qN-dKh6VzO0N&A)wkk=1m5Fxl z{IzQZ^hOXBytHyS+uBJLTa`y~W>JyVFp=!rl+b7#f+$+oH-ysr05nK}=5*InI;VeSEs?A3+}5l|;y8?YArUcvj|Tbu~G#stqmP4s)ro^u~5&FlN&t z+@{&C3=)q~{~cJf*HO(5jD(f?gb$TS5woNXK4QU(T=bkz+DA$o{!-pYN-FCI-fMW1 zHiTi5|f^IyA%1 z!`G?E4|IgQNk8pWW^=zVK1QSnSeI#X6{LZ>w+wkEi0%(o8Evgn+QJ>($tp-a+bLz2 z(mCn+TSDp12h%!oOB#N;xCk2dZTWcbcFw4+oWrtBj)bt*jc6BE#DSE&TbU@2Lpd6F zSoNoOb}PZjSu-#qaUxPA37e1MsG~jzw@UEMh3XSi&_sdDYkrRM#?^x%dhS-@LpnVI z7aU*@ZtiV%2Y->q&jD8jdmyslX`4NYUr?HX%SZ#4I%qyIIt*Oi-lNoc^#c+ZHYs^P z62ig9${x%iWA>s))>GYH#SiZ}T-vMTSjHfq0RieiKouJ_%R|9MgifsT`6sxwMi!zH9!q^z;$KfiD zgB^sT#(Wl)wJkgZvrV|ry#8k)<4#nL|IA_f8CU;%P~tu%$!jaxR%X4L&avD|dHa;G z7R&Fw14&%{I)2Q z10g;EAtrPav`^9a6L%*XE!&MU#rTz>b=(SU(v93hxD3$IXerbQF2eE_euq3*wM%lG z1TBX)dTV}nq10_WUnvITs9{jW-%|%xH13g&(-!nQ6lFfftT->uYADK9-*Bs#*AJc> zk;Yvel>x{vwDdFF9bZ9PJ_8dKkn*{b?$r%MDRcu-Kw}KD&y|>-5g^Jw+_|uB>Yy!) zw&Kwnc;bU+)WkvEw0UuE6$YP)?X}6A5O53F2mhCF_pKhD`EnT^#{sA(Y18M*eXaBS zwZxf|uATxG{jy`UKKSMsuA3v)^o8APA`Y19-x4a;@hL`>D1E;YA>Z==h7Z5Uk<3?( z|I|2u%ctf0v407{Oh_ejr3CtAzv3f~qD%X+-htamX}e* z0Z8e%VRt}D7iUoNm&ye!*7mQkUrM53UqM0(dySTTrL?pGjdQtN1<;~r1{Sh3?oY$5 zg_3ar6Mik$N2>iw86}^y`$&{otqhZotu^t$98j&q;@V?PHLCcC+-j80Uf%$i>-cI1 z^lM5yhV;xDWo*E1F7g>CSOJdBnCtin)?Y~GqSM<6}y)+0N=bq z!+2VB5KMju>5n;&#>Ud#gG#&<_B#D>Q0eE>6!>TWH#`Bc#R|1Q+%GQABbHxsGwv?T<4)N^*28}p^o-I-v` zgZZ-aeXWz^dsvByG0Nj)uC}|N9paQ-;DW)-w3WJ3EZAa=b| z#>>VdA%2)9t|Bffzh?%wim+nczX*xk?boLdWq4WlA4Z>V8Rtcxo>F?SYA)uTCg0Oa zyKdD&ZCCC7SFxKwX)f|@MeSceylcPQ+_Kz2*bgFKQ|R>#ja&Gu(O0PGv=WL-T`Ny3 z!GWP@zjpPB(!)r~g13VXz(#n5?mw%=$J^d;`n7}d4JHJW~W*+~+4wdh5GzP+H>#TL|X0Tb~VvR{N)w4I*6sC1F= zhQOYSN;}Egmabe>-r#RjT~gAdzi=<=l9CO{HQ=(673XV%S&sQ|Y<)Z)i34LLq8ej$ zi&xBQU>Y^+vJ!zSc*`$?Ma!t}vf|fbG?viV~KRxzuE1E)OO&s_EDdZ7vnBz8{29hAxD=0vp(wABy@rI-)!9CiWF9ZV%C> zE6Nn9*((%zRS8u70{c`@##JROrUsw2QNlJu{)7te`mCN6aG-056b5CA!i;&>wg$bJ zW3y==gIvW`VTRXU?zNOlsOy&qcwC4D)5lj8|K^YqO3e2REM}{B(Mt_VD9H`VQ-1%G zm@U!b1|{5!Z})+^p#1=jqO1*g-=w-h2^JU8$p(!38dCnk+IfPy{-uP4tY0qpkIfR! zuY--An!dR^qLtKye92v+9}> zgsgeategK<{N*=63>s(^6X+s>T>X3lk(=H2%6W%`**1m0{K} z23GBL4lt+8Lf5V<_u;Enrr|e~{=UBeDmRbHD>X+xuI=A|WYX^yzJU)VpGGAXXMI>r zIyNdvFi*xt+6X?>sD#<7t#whF=(~D_72ITnH}R8@Ecsp=vvW1yeZSSH*x+p-q#NyT zR0aj1MeKc!PW=AN((h#n++D(VZOZ%{aQXSDFGiBS-515BEro+-t*#qq+@9qls z99@F$%Q9HMBLutPe4Dl|zhgxPzBAU^G<{-5#ryf4D!vD}k$advR%GMF=D4BD(D9Ok z@5;E3*I9H}V!acxKrI7LT=V7k5fjChS1ZJ;x@*?vv$WB~9u~H-S&1EHJhd@`#^uhK z`$&>me+BznRilA#Oa5*yXH!Y$&Qf&v?sIu|s(y;7?OXxQ}+{)UD zn4WW)acIo`nSh~#d-HH)NjM_MI9dm_^vDu5YbjQx#8{&??<0tCs#h#MNI+O0jYz5U=V)(pX z!sqpqNvxu=Gaf@kADp@%%;fxY5_5e8c4LWkosf_ zhp8ARn9T8BjYWHK9Nrv`XBD5J9~1Muon@S`#7+ipo=l>9y;*a-^f%d?MfR%f2Xj6b z4t;^cN#KC;C5d&nl}EUfmc(&rW8!l%=@u^2=zuqihGpf|wsgUpW%!-KZmeH5!C5d7CIp6px#mgP@45-+ z4n~P?r=k8F8nA>u_hX@n-biy@<}*}_`82vM7dDv~`!q`ff0E`F+=q@xn99i1Y8{sS;&pZj|9*0vD zz9JV zNrvxEApMr$7)3L9w8}>-=Lg%11;eYr1>`B?^%Z(`(7r@LF0b)O16a?(!5b0BS6w3` z2ed;p0cGKxW&k=*A4afLw=!how#gaP5CQIkAJj;eDo;SYRM-i=>@Hw^ya<^O+=S607AQ~fnz%Sts7=ezoC>%(fb?y7|D`g7)RZul|e^hZ?s5!Y-u2_dgTg#hM zV-yR>EJwgz+uU%zn2nYMpe4BX@ZW%lCJ-+J24hu(9sS5l|U^Kx`~-a<%`F ztcE^9+2F<)rD7fs!)mpS>+tyg@eM2Rt!$>hh--nYWx+YzjnvixgM7Sb@H+URxXjOx z#qwa_@FlPb1*Uxf%UUEj_$xmK)_0Z>{BM@og))3F{eR2k@-p28{vy%6sb zKihK}KxyxChE|H6>abYwPsER^-W4OsJ&r~2GfQsZ!*k8AgW4$3f&W>ZLkZU%@ zfq}lHnm88JGX(JFNx}jv4$4kig#BE_B#Le?^4X|%6gFt^Y=c=Ch9XcV$KbRN>0FEc z1=5X;J;IwPIi9^*5R1u;cYb5)9aN}=gKRM1aV+|L+jev=teo%6Q^hHDE6>5EdD=Ru zSqL~oZsqx`X&##`&?Gtj>+)}9s-iIotWCf#AfXFgVR0kzLp%7TNnkC$Iuo za&%2(smd@kWF*Z432U!oiM59AY4zED#!C*tZlekFlVO4&g) zwml1NRTyN@={<}nP7L0kWypJNq1m5PP+KVA3$PTwO0!!6_zSeOCCi^|LJ8LDagXAk zximyOmTxj#CLnNCeosKNHkmU2wJcu5c9m^Qc}XlN`;TS#rb(y|#JivRr?8f8mHXb$YJg6;QZGzl@C)?{mA7Nv zt&hxv#t73G4NQe4nrA9Yb*~n%a~V4yq@-0ttugdaD)aP91YJas_789u@4&lI z6h)3y*1Yu?q@r^+bL+gH@XaT*T&E%g6JZlYNPr`7KAlQsy&xy2q_Mdn$@BPf%nwBc zZL~4};jRxxFAnn4SeS1EK6x82;@^hNZw*~fV|NGIWA0=%EX44f-kt@!SKDoHq#c9p z5f!xuCzqq>Kz&tvHdgf7fNy>TmyIL1d&n)F=0*A`oki2{4`ENXIi2}{ z$&G}N7j-0b;+GpjG8p+?EH-;_1E^+nk3F2xeFT&@OfQWm*BDhKJ;N48=>)~;&^vcJyfD^ z>B6Rp;$~{mjSZ1kWAdfyu2Arlc4J{U*T&9$4sGcMfo?7x>BiQ2pFj<~%fER6tnz}P z;qc_r5AS5Geh{TIA$=NO-(|9>*b>2MXA%X_Pr`f1B)B+gl&q)Au>ru+TZu=;DJ+Zi zz{zx86j$p6(#$Lt#%B{ft9PNUH(i?%84E$HGsE_xpz<#LG{=1ph9BEf> zrump@!JgVB#GStx-^`3(=xM-HQ0COlG;cF)tC_~@Ic}sCpt83-&BR~K#NUzdgR9_M z9%fpxm-ap4zPTCiZpQC0-%x2p{8u;A+|0Dm{7Do)$7RItyP0Nn!ZXuyD4;tFget=d zo2%|D_`XWJ&4gHs65K}Pilt~pFi2vMqmP-k-I>;av`|;t?`GO}&a`Vt3wNd2ylHiJ z21isNHVx>(f;&1%8}4`>`5(m3Ap2WR5`wffh#P-1-bucNp0v6LwpM`SNjptA2GznH z_apB;uKe-0sI~=heQw6zq#8Br$%4`O293fUlaQm+&0I}s6z<@L6h8!*)VPdR`MFwU zZ3g;=J4T@#r_@EH1-Q}-Duz1-JJbG`is6prKzg$$^F?ER_oD4Rjb_=r^`ku@@WF%C z^_Qyv?8`#LWBQT4EEEsptQQ5EayJVhpS#%zX;-F>!1Iz? z#D#iWynf{#1}9t25KSD&Qf>xT4`e-Y&hqO(HpGl8YoTWjVuziDvs&n3LlN?Q4xn_o5U>rA)#Nlj?x*X9NbK+^iaCQI}iALPVwyQ?g+IR{Z!RBK>^2P{=)_6-|1Y430 z-YKyU^wiG#n3QV5HzoEjytUJa=i4m4xLw6;VeneDFRdO4orQmM+BuR1h7N45ed{d$ zHz@7zqV#A#Iy(~NDD@+cQ7l?o*qqvoVxjFmZmu==6-wVj8FNTGdC134^F;i5S3E{v z9#?%7ChD_Bv8AHqUqsEvuuov@c4Q3ODe{Kl%qosnk7bivY>8izyLT=yKVTh-aOtsaT0?yL)RppJ}92i;&LU;K7$<9>f>8AkSk9 zlzWh~KUL(hDQ?dd2&E|qw$&rXv9*%ek9Lh`?}k?A*kU#QQp%UJg;E1AmKY@J;W9AtTV0Zl?t$KikwI~D}{^BX$%#L+_^BpVSCGn|~>(!s6A0J@dafD-^ z$Og0uK`yH-)NromITP@miSNsnc`LAtRa(7r8n~urD_?O|PK1b%Odn2U>7l$}F3!k! zM^@t-=Mw-Q=csl$byN17-DJ(U3 zIGV{T^hbqORJa-|gHdVtLu_jE%E!jR$^{}ZpXH>aqQNP&>LK<-L1otVUJX(H!uj2^ zg`u_(VRHpQv=7N?*eL~_&cnjOTb_V+@%Y3%AI-fVtN~)(Uro#tH0-J?`-!1irs#e; zD@Yh>gtNOhj1_}U+aU=gpd0xf5Mua2lQV2a4{UfEXErMC$x1CvV+jL5fD^#v9tHqj zc^?44CJ`Ayyd4iS*p8L%_R;(XRlHhji3cpS1^67H>kqTAf`iY%`UugylX&!(2pieZ zY##m`Vcx7-MHspn5KE;{*xQHzW9Hv|NBNKLDE|S%1(N@1NFuy^NofuYR&SLr82s)X z05;qKz~KS_+PntY?B(ZBw1J9nbP8eF7Bp6BhQ`1?vMNFlokF(ys_E}Jv8O+4*m<2n^>CEE{%;!%u zgB`|TdiV);Mbz{OPcq!BY^#?)#hO|8`P?j=DvZXRO4cqCUSC>ilhDD|Sg~R1(~V?M zWOtgD3%i<;l>H2g8I}ntx*$+EpEFSy3K@H|l`~(z!h@IMtgjk90A^T*_mi6R!r*o& z)07KZBixh=TGF;>SfqP(4xX|oLiJav?it9bp)={PXD}tk%+w=ivs1EoNOyYf!*&NCCBz;rNI!V4y>dl{L%_V+} zKNlil#Kjr>0K4!Yd=&F>P?=L3jlSc%Wk1Js+B6SmEG_83JoX|t-m@7)lwn4?_aa;1wip%h0p5~kEMTL&2usfuP_z~l zeT;rwz`{}WdJD8cWA1~?|*8#c6>byvD36j`-h$OG5G3)v{&WsjNL zw=K`X%fshU>dtVz`->sE=8~j*4m?U#Sbj^vtC!&l$3X z(Ob*dA~UHfRPRh+K5>WcyBzx<&E2b?o}AaURZx+`j_TLfSd!gis^HJpv$`A{!dX1DGwWc3 zfEm0FaZ(GoK%$h@Y=LBdgg#o$zPHp(6{x@+cPlE(*yA|dX;IG7q--0FD#zi}i!{HS z4NS{xrdeZ!il}GAib+`ft=TrsI#8?j3davi$ZHJ? zQ)W*!%)-}zeQl5Ez1FaP78R=$p5jWTvumV4PfjwSHJC@xj_TXqV6&tMg8;ca&gxk7 z5biy3f>fE|`7~%9J1TiSLai%UYq7mPrULsWlV-JTP{bYWbaUX13+iV0k z(R`cfEet?~JXfHj@j>3Xd~OIdEU{z2t+DvcpfT^TB;{AsTS|-GVXsKVc0GPQ?=@U; z-@qOV$l_|gb#@KVa6sHv6QSlSpyxNTcq#lL+OUx=YKf7XV(j{iyzqv`wJS#cT*M>4 z3r266Pzmn*g%(w^C80j3)aqS0*gneo>bhey4@tfO`#Fy84^gL0Y#6}W+5!0h;~L** zXaLtb8ldWET8$6ux`P1KNllb9d&C+YVJ<4*?UXIRZ<)S#3tJ+JDKtViv_eO8_Cg#?i`p5t_W~#X zJ1Uv<{8k*?9G+}kEvDbMvhh|GsLzbb_-UY9vc<}BkhJNUzJxITPf(br> zluy|qxHdTQDY$wd1?*#w!tA_|o6`9^_#)*oftKw9`((${&V6iy$5wc_Lv&6{!GB3% zrGKJ<;G{S$1_PU$4L<;EK7noDujoEL2dzODa&zIH`j*d_BzadB+c?!LHamMlQ6Z#v~SblLSWLFK#l%f(yItVEsFoEI@vbSAjPSTNstgWleQSv>+ z&NatYZwW^6i8H1qq5IAcCD;`haAe4a4AKaSICQI;E&MuS||c)I@^7AQ&a z^u#wTvjAhxypRL)Npo`3Qq9ZD03ImPZtOo7$22_XA3PKdGd2J_?Rv!hHtiG1*TTG8}fP}FMmp+z(}-p z0}|Y-A=p4729+4`wcWHbDRcq97<>Ymjkv5#BaZ7x5ythNTMgj(z~p9Gp{qd~BM)AT z=T}*b?<%P4TzSee3m}Vy7=+pF0w`E+QiK!iI9xRJ0>(z+GH!&$udjtVa*)>GDq|D1 z82J_=yj+H>Od;PpnPtwn^4IdXaL1-bz1?^09thof(Glh^dDinyh@T@GyscCDQ5KZ4 z31h4b$Cnrmm3Gertph+CS@7>b8aGYgVJ?Rq!_azzP9BA*;FCa6Kd?k8B7yq-z#?E5 z0{ckOr*Lo)P5%Kk`6tlaA6R|cN}){A4&9Fxu2>x879ljxoFGh3z&67gG5b(o)R|b+ zMzc?zbagUlomd3b(2r~fD=D^Fwb%8$pV+;!cS$ks_PYk&0@{0w#o5u{{NBEsqX3{m zxsu{q^KL)qv8NGdn()|;v*f6YaawoC68W|1Ca09SX~%FEzZ~L9Yo62gJMQ8Z9%o|$ zijy=un$3;yE)M8n7~;us^wTd;-R^3q0b4w@m_&`gKz(zGl7C?<{r{oFXLSBJl+t$= z_<$GaPNRQigZ`;h$v>AWqU*mxU#o`d@^l2 z!SW^lSc>f={pdJwmA{V!EWs`5LgC_;id=# z$I_b9Y>2chiLRW+Dathp_?>leRRs$E&icqe&q3$^fL+?w-&ru+VbuK23VD$sXCOA0 zQPCOpa{#y-f^Iu45&z&9&E`9AMLx~=gAGW^Kvc$_c3LO=kBmvu!q6S<@xA4RB&`b$ zm$-T<9pNhgdG-&0%%F&~kZ}?y?<^bQZ9|gqP%ESHJ~6cWEF?K4hWZno;5ITS#?8d~kXz%|Q06 zIEwm{wX|MM((=p>_>)cHdY(Taw{<123)nX&Q~wLBvlJOkFJ8duV0tUsc!9lawSl?N zra!3fMfQ{pgfIvK!mO5r^EPqbiUg#(fdY1X@1welY-j?OLf*5R>_TZ4)N8!|qrovo z!?6JIRl|-S8gvPr<78-@Y+W+h&`EMqFSB-@P4Z8ro|nPyxhAi{vD#%8>xo6o1#cK! zKd>y$$Sw#5Bcpv+*h)7iQvP;&_X>M*;L0RpRaz2>iSx9Yo(CNHfOrN{jSb;kgzX?9 zF7z7t<|AM3smAu>u_zer@p87uIX;fj%&RP%FX+7GM`#s7ZEwyieK8aD!8Qwi-R4Nz z7d8{dZj1?oCa;^_@(G>1%3`H~sno221zIbUv^v1rLdgvn&p%S%1{Q0rHplu}n$f_b zQYIv04yjcQ3O|P-VLSAD5quM`?!YStcxA)ioTy zJwAZ;{>27b3r%W+g|D$`iMMEL42Zc-YDli$b@rv7jo!b;dV4~uaLqn9$n9^IaQBW| zG3TQ|`sOBkjPrFO52+^wK<9N62f^jv^yfT;p`6^uT%`K?R{1)b{WoT~?~=4#Ak#D_ zDmQbYgI#sk(1)nodd{41_R<-IJkhbP^sQvS&VoH5C%IxaQqSvb=>M?N%e3t}d&=|c zTScc+*Bj{fX(p8>QQi%9KaOIZ1MMFA>;`)?sd~!?amJDy03`jJ2^|8kzVT)`IiOmQ zTN!;F8bvD_S$4B(p(@qbeekuKMbW87)=j*;G*Qh8_D;~aO%-+!m`?+YLuH<+c4?6c z6Wpb%)GmA~#Q>=0vzBuz-MzFyjdcw)9d90}u?aSF2I5XW-sa2=>ESk%5?UE^#p^U( zRI@zMFRo#x(TAei!&+iC`B`$4)HpX7;kavShq)HwyCrp%6+@Q~^B#0wQeCvU&$v0cO^KvGENUAmFP5TYwUZ}g7*}7Mpb4`2iiC||qpSvcHd)coZ;IMy;Luh# z^RA7+fbcb2;^c&Cp-I2K%F}&>^q>2Q(*?*cx3$>WiRffSeZ5r^s1Kl2t=k{#d`IP0HQYHOj9xS**U#ujs~X!1FFKW@t3L_{d5@|cmSX;CGHk1bCd9&g z<_2cG&txwLjc`+K*6}7YV(oQPr{EQoGj3|I%&9iJ@qB%ASEIq2x$fGNL%^Db+a9ZjI@!%RPeq5*(;jM!SyPR> zB}(!Mr`?G0y`w6;l5^QZjg#;=PNb(g4su12rQwkN z+U2Eo3CI2hQfp;?ZAYz#u?^_}wVwIL-E$P;t+w)frj_CHU=OG_OdsT}c7Pv^;wpE2 zK{K^dcCRe1m0_VII+FCB{;I#Fz!ZZ^xEO2zEQCpAj9A0P;G{v2&jrf@{<0C|Bj_jj z1wpIhgMsj?#KXoW*3I!g!jpt14z_q&VY*g{cmrk?-`eoKg1@ol;ggmTZ}ZUdT?t%5 z?~V%11lia~;u}&vzinm~=6fHmJTv0UZpDv7{BZzJb(PPxXnc=kNur4%YTRhVuZzDKXISXAM0&q+kP{baPr-zK-5dc=F<2_H zffm;}@a3IfY}nKs$`_BUZ5WejfY|}dJaiJt2HMX-kZD0DL(~{4JdnIY)u7bMWZs3? zqB^^<(8GXI2fL7`KryZ*F(532f(51B1G(|g8Yx)qgATXuf+O<1hN=|y6Z}X;(AL6; z7aTTmfT^_sdZ^x4gQ*(HjW| zZW#7?aHbfp60D6!M5wJ94ify}e`8SuAQaHv2z83rEVQ@?wjSBI_VE;Tja0+r>4-Ab zLenDEqg*Wx^HVG;e5=`sMntI{a3uuo<4xx)RTo+nrM8YL5uijf3G2gI8Jq(fMC|={ zK`G}A^rO@`s|Bzd_eeF=_8mL~)7GZ?i?!6#rbfG}F6cEeB5l{Mv<~BLNbe1@EIxc6E424wA3rLlKV4vKyUQcybrAN2^Je z$3f<^BgBA7epa~u5X@U*#&9b0N1OO9F+ayNUIm6)fSlEk3R!6D{u{&}{^TQpFM;b3 z&V_RYZfGT8M3~!ilSX_aw+?->WTa5B1n*){WpBI?Jw95+)1^6O{xmmQP4-;?0Ioef z2Uw`@jaIuz%?GBLG{kA0K|>*c{935vq(RUTIscEgz(j(}Gaj@!PPM_#0t;$L^g*E-{C1GRF*zvhY`>SgB73gY<- z2ETGMp_vmV<(`pAgvsTzJPcG!K&b!yfMht7+VWW z$3c#MuCT!K@R>5g&#}c-5QO9{-gGk@tS6rca5vnZI$ZzQ_6qta9Iti5(VG!ziFJ}cV zLe+*Fgt}tJSHhg#Bp}+ChYHRk-Xv6*lTZPU9k(IwZZMN>BGRPi3(}3QuX6z?PRF?)%(8m+*I zQ5aq8s{Scu`{-x8sim;z6l$aNxmjwiR8U=P81Z8Ai@9YxeY_`(X0g3=axNf58U%jQ z6>oM3j9ns)4w>$XH#?*{r^?JfX_2*ev ze3~=9FxoKGe+uy?lCYR)g5*Y5e0j`GBprxrisYL;=$_tc5iD?b#p;K8t8K-!YO5Fv zIF_4@{nP~~@AmjzE?Bk}l-^gpU%C`e)K_gMHN?~YzUoLGj_;?&-G39`76!g=ba%Gz z98L*1|J3rSp>XhOSNucqW_%XZ2{>_Fj(FF#D@ZxGoflyo=CnlFez;}zc0aX4q;Iq# z5v^My)Fg5jBYCL^XI2<=*%hxh_ERNMifu=hyVaIbRx4_AxB3YWU%6ZD6bSiSLPgwa zh6^+z*L?iaihB1~e^h_R0n|dAza-tGo>b2uus%Cp4;!FjNJ%o?0vy za0|KLG}CJJIV>o3f>4TIwH^GNiuA`M7~Ug&4$ilE`Y)IOze65WSZxn(O+yB$y+XfE zF%~E;IttDkaYlM|3cWW-eOj8mM{hG&-5^R;ne_7z^--AH(U76)2kw(oG&jESosmL8 z!_-bvZVC+0wwW=cUkqd)2|A@1$tEfex>-FxtF? zXH;2|Lfh{Jq>oeR%Drl&1QddXtM~k0C>9J?A90`I#NUmSqee!zPtev`nm@%Y zWGdFSgRpF=;erJ}l{N(J>;yV7M$L+?v}Pw5+xT^J&0OcO=40xr6dq6DOj?qY8&7xL zuU>^&Fc9iJ78}XRL=&Y-CrSbId!A~glCf&4nFjPKo#=Jr7&Xg9%iAZ=;#|-eHE+yS zZ}8@A%)8mV+h7$r+voz)f>w@G-Q0U7@cyq%9E6hN)aYK9;4OAA>Z6eEe+I&Nm@ht@WSt)vrC|k}dbN z*Y&Atgrr&_&sgF5{pfV{kAGpS5SxDbFTkS8JQ_cKz-BO^A_a9xEQ`Hu;g=xstrX8FlV+Jw3o`pDC-xlV6*8CiC zuDHS!5Z}+SIK^3})(kIjg|ph}dzPr5OYY+mwO{}Zwxkj*15L_t5qg2RZEvZk3DSP?7((@~GV?kFB&(TdFo6 zT>U-mDz}8`Pra%JiJs>c_H@^bdH;D2dUd5bUOv~)%ZDth)Cb+Yy783i^$%zA6g$1L zO6@DHm+1H^bYO{Cw5O|BTz-%B?kK~dMRZ1Q8ugkQ?pYH3{GP5SVek+$e-F)hO-+M; zU%}i$jBR7|nKKLvgxJEt!|#X_LkkBFMx1~_C6sm0FTAEQd892s1A|~bN;gENr<0w4 z5O(cN-bGv=(R+b_`(duUB4p-wZp#aQJBWMhR-8U$tvXM_izrP%y?iVACiuT_c;Eu- z62~tgu8-icz$*IwPqlv_%?fn>DeVN(a9Dr`FWm6XiGUMRND8}eDJJe=U$Oqpv z(m1Ri&%VyJ36PhuQ+yoXHp)Jv`jO{OwHQw2wVmp>(&TUG+0WJcVKo0)mHH>%N7%7T z?I=Z51~(G!1NvWu5Z8dca{5|RFTo+ztxmj2`CDr}De z^fUX>o`4GnO&EIFEg0*`_e=FvaRROX63tk1&@7pu|N0g5j#9`=gz5uzpBgn-3RqXu zNRxoWL#uC<$ECbO>RR}CtUIKhkOtMz1}|$HYWN!ZiW9F<>%(d{>8~g0k;7_hbsdB- zD{d*BKB-1{?R)J832xX;gAyOnfy3%3+z?CrM(y0f$RFr+P1!p(6F4B zfC(Zmv6$w6qxOa`?V4}Y0qPn6&ME9h@!zTuz7vtHBZfRboaF+*Xd3n{fTz-?Z`IDt z)*@>@ETdk>e{pVmy7sMlAXG#VZXEG74rMI(=0*Abs9_aF-iy}E|6TnF13F94rb163 zKvTFILMsnXL74lSc7-P*-1LcvbzwEh@W$vw>^7YZ!9UE=AqSL0=7VOxV)O4 zKCV6;G2=iZy^8uSVW(j*`>PzbpD}Ebb>Cmq0g`%c$JDE-CfDrBpw66YJXU}@9wv1}+EuR(7k5&~Nj1#Oh8LZU zY{!=0%Jv;iIH~3v9=}}K>N*-c=>0y==H$V5meZGq)lM|<6!voCY5FO(cgXWdck#ts ze7pE!wqA2e^$=Sbd?#dT7eY<0aD~qz%(=o9K8-LOx~UyQVwyl~(F9@`hv*|lLl>(# zff+1?@FFKLXP6e8!Q56!ThBmU4+qO<)Oq4sea0W^9PY=x;jG%->yh<0CPF{wcS6@b9&%e|J-RDm=SQHgXynxB5QeSvMy-$+2ExSQfIK*Nj z>t*$EF&9VE>Lho(uV{`NT#LD^1~gm4ZypPR8JX3b{I006ld8GNjghUaxmK5ttXbQYz#kv9N>(n)mKy) z1JY+#aKuou?5cXs?WH%-N5=*2Lu$B+`RCHZ)Ve{9g!Z?8gPP=IBtI%>P(x!~O%eRf zdQr-T$s!qb8MOCL{$Yd2ocM7HGk*r+8;1xLMwzP??z^-)-xkEX6I!2;rAgMdV#(6c)-c>) zhXzf^hQSSbZN}2)t(mwbb!)W9E(|eO{k3H2Pu6?oD$KalQMAU(=dP?#^?cFlCW>#+ z!;-b749St{uS(WnDI5-EtXh_0zUh#qWaDiIY5j|!M!!5a`bxHrv*rdGZ=_A948@w{ zxf&$(!;D;ynUgjzUv^Rl4dwIXr?g(NMnfNx7jdgA{vU1UA6MtxKmPZ1&Z(|* z*0#3OR$Fa5ZEb5?+uD|C*0wf{kS!Wwu|{Z?tr0?;5JIdG@7zKxgpgbJE$-0hCWO#U zHz9-&a*Ks*-{twZ%Iv9`wT3rJASXgW6BTU6OZ=IC#t({q^ZSoxGJOy!Mf}+W-ivJq_0D$==n3O#Lc8qjLLUBu$^PjYwZa!VCThHv=X0m}z^qF$w$T8V`Sc(DrcMOX$0J zXTyE?#q?Nj>pT}-m*h3LF&IIQ88(+2lNAU#883F@G39Jo)f1u~LxA_#`4uegh_>a* zc2a6b>(>TaQb9zlJCMhZk2@hUwD^Q=$$GuKRV6Gg5GFS-L4Sob5x>pfv8co*si!px(m4si(%TtR-r9JqoE z?3SIyBHeI`Y3k$&<`h@@Dr<(8#OW;)%wrej%oaO$jB9*fb$OFv>fzz%SyKNa|NI7> zuiZY48|f9(!3#ZZ)xSA7*X@T4Iu8HR6DGA*Nd@omcl@EdJqibT+Jl|vr z)?Z9C=T9F{g}jJ{=3eKS=M&o(lC7m%lh>~-ben68Fnzw;Tpir8u4LYxrTR&?IX$jp zotnRA-Y*CA=#ur3yty?xFp(Pk6Nv{(!%a6L$*Yn^4G!O2lOX=S@3|6%(htm(Q%NkP~Zmze*EZ{R2W(KlP z>o#-s!+o=?n^0eiUFHACePE=CW;HOX1;67ew zM4BN+hdg!9huPJDOP;%z3Kd0~Nj!e}WTd%r%VMT(vrZ7yYtcTBXo)7H0MKO1dcY`pgLl4&MNnV&NC z5i`uwoW2H6S+M6URGW19Yx<)Z=JMc<7@Sh|6i>i>m_ErPLm|d9e~(+==`nAzeSMRj z6JxHM(sk1T-R9!uJSYiUc<$cdom8?IzoV~;p*XMU`(n(wJm>RjjH!*g^wyc?nuRwJ zN1hF8;l=LORcyZhd88?itQxB-uVim29U^&2xy`J~q$GtHHAhsXO`g$35>8xwVN zm-$|G{ldXSzD628WUV01r?<>9(-73JE19n!jWwrCI)a<7)41Boi2&(~nj`IRZ`c2h zHKWGNW9FMt zJllQtd~>_oOEB4j_XL`oY<^JEuK zeAI;Osn%DlG(Rz-_36pxnK-q0_Ds?LOg2x(ToSXVQpc<^_uBTA>U)kfr}Dtur>kh8 zM*Zt5Giusnr3ZB1=9FLClfgF#Wn@37m#j7q=e;P~R+~{%_Yg9i?M)ot z?J$?}Ry{d4$Vzt5rSDm7-fH_fLoZ!pE^$@F^u+9`Im`C;PJR9wa~<}SnLc~AerAoy z$8%TgovA-xW8Tb`YgY6!yt?h!o`d=1XFe%90OwdUyt zuPWWO&V1C^uBRPo9=`Al!imy9$Lbg)NVEKNbkZYXL8|gH-yG%rVbtVAebtc+$>a3h zU`i@6`yNgCwNC2%1S`VNSH8QBo?NZpmQaW5?*s|jnQA`Gfv0_`=BzObgMIU=LRQls zQq9OTs>vvvW#^j6)7x@$ZA|g?Q_o6L47OelRfD_AnX+eq^2l={_uFl6zs?Gg$Lri! zS1Q(dV)VxK<|^YYee-%VV$!pwT;1iCKjM)tF_V6yer7$zc~1Xpy}2^x@v9H;grk3r zppl;lIXGgM(0Ax1N10L1dkNNamTk{1`lO?n=9T(V!A<&Z!PR=-QRW;cEjxD4XSZ_~ z?7Jb{Mii5jMf9*;qWj7*>5;8ZuRYqd8L3DaChhsaK5~EQv%iPke}Cz-zk_}Bw`2{P zb!L{o>9)rEE<|H_xgSqY+tr@m2`z7F;e}Lbro%&P@uIpTkjKfFjMuy;>!6&GV@>>3 z`~ugb3jg(}LOYHzlR_^&?VxYK_=2PzV@|UrIQ6;5n5l@}?prAPk!evwXG+7$WBSO0 zY1maL#;i{B1(+1Vy39absO`s^GbjG&-19Imhkkf3uWXW@{dA!|{8%>L73NioBt1)j zGWNAmiSqfOW6jC-3Ui-6u-Tj<@3B~{UpbcLkgLjZ=8PHpFFmNyg2?k}zT06r0wx+E zUk3NB)@zP4XFE>43JHhrd9LE)%t*u!w-c2s=HHGpSMp7>ACEJmW-Uf+BR7b?64$KW zxHlXLZt$+5)n@ljCK1~V?^95mgZF%)R20duVrd49>Q zrZG?6`pwe6V$ZueW}~@a#`y&68Lyt{TZU>tUTn>G!0ULux37WMhU)Db&6u%ek|X;# zd3)KN`p%8!tYtibI3RBh{CbwC6nHpLdVC+BgrJST*(=*WYybcCUgKe3`*INIhmGb5 z6J}C%pHK;1lq@pS^$8hf)P#sboZb4040AK@g8X-ec{yiBPddST%S?^uxqmMAzL)>% zloL(ooElzEMAp8S6P@+vaRvifgNN<5Rjg>e^F(tVd-^+1(`jpq39zZv;XN6vtX+B`g)c?*jZ{vyc^EYu9%EB~F-?qtI!iMgjo6Os0 zym;xsh(PQ6+WIQ%oAIwo8Yj%tmz-oSnbCMB1rJ&K*BN|5bSsbP1zs;_t!51Qlgw39 z{(9w~OGzW3-_Ta)SCeR-JWR|%M$>0}R=6UjHs-<9;HSIkeb1LJRV|?@U z?o&-qWP`bOz6?7`BATW@Q{s+&itahpj2cx)79j~jBR7~=&DZYJ%tcfANY*~Y1W%8e z0?0l%?!H)WKFy4!T@**-yJWR?n^9g$vtUm=)`QpBsnJ+!#HDDA?<*y28CLFbMqlx+ zxoz8!Pheq8-}AZKc=#CXa)h&v1-6gNrs$^vW$xn@!lKu*>|;6V8RFAt_08o=C{j=( z-gisP7o5Ut9D3$vGh#Va^+kJ$Z&%2Slbwo0`>$m9G|Q0+cc+!izIeTJv$=Y@Um<68 zz*yf)o*zcg;mc~@LjC$?GXb@COoH!{Ax{`5=%{S-oQZF*8d;kIopoEbxpd-H|F`@5 z*}gRP`91+MMaQ0QMvx?u*)^y8Jh5}BluspYAgy#`lr*) zBmPIfwJY?IXZ$aIh>UN;FXDenbL|m(v}ZPl2XlE4x#y<2qdZO+A1FMtM9w%KX<+=D&|JKRe3& z{3!E_qs%XlGQT#;{N^b0J1O2j2hn?@JU$p@{&S70ys9^6B9-&4x)6 zJ!Za&V_iFV&@ki7Je83B^n*p_TZR+6$IMc38Tx|D%vSFe#;(}N;UIg1k=z+%?>EfW zAbX1~tPZoy(Gp~@vB!1<869@BJxF!?%;gR-H#sDHMUaH44@w&fGD?EYHm6Y;Y_>a% z>fl23C?VYj=0VeJa2i$4aLnz_&~B&URmK&{`ti`ZI(XNq#_`E%19p3}k~)418;vwEdu?W;&1klnHNi%Q&8!VJx@@G}ZtpfQHw2kw!Nx#P zIOf{m(8l2ZPWj+n|9is!kLh~Xh*efW%!>BRLE{9QP(0|po zZUb|P!)$gKwT^JiU5?N`$A9(T!MjRrUgH-d*{q2+GQw-4Rk;z?9Hpua6LX{CXp2() zMp8qRYPN~F+m=)vts3k}?NO@FE@2pp;t~`-98JT7w@0h?V7A_BASAqWifT7A>Y`PT zQ{r~I!Yhd7>cqD>G_rb%y>Gn9%dxpyw%Z8R`Xg1@Br$hPO74kN z6_dleqEzGLXj1B(oQ!)zgfGT22sTRXW@oTbZl{Gy9Nh-y-XOCn*r*8($J`kl+WUXakO9N!HR8j{rWjdCLsK|J znlv1#+6{9kQVkkicAP5Au)#>LYBs|I8JrW$nq(wim?-^y2eCyMXG@@#Y**u zhSg41U1KB5BkZ-~Oq|=sg>_C*z2lNvC#&l5CNJTv4GZm#R4rjiBrqJ7R7+wLB}}P1 zv~selaEB45#Vy&_geR9xRz2Zn1<4;5ZkFOb$s}miB$LRslfsGIGASH$*Q8L~`+erJ z$tA|%6r()CQO;xO5y`}^ifk39z~FrPF0VPE{ON`<&bA<3GQpX<@}wQxckH}mhv&qL zE-dsEY~8W#EYG$bp44p@ZeP1$gXh={$DFtI;;oDGx9&K1@rf7hDBO1b+3SnWI=i57 z+s+;Os|n8Mbjw6%k{;uB9=S9SY1@tq&%VH)*!Ht?J?HGaz_TsabIgw1vx_|WJI~tg zDcHH~!ks(L&ea>-&Ql@+@h&`j>jh`!S%n~;e!%TKCq4CS65F;lf7@kSB~J?HOZ=Rz z+w%WR`hN-JtIntr=z$5&2z_$6GfmfoI}6N%{+|w;b=ZV z-7!kVUv8=z-awKI)g1B?a`P5eYxnxiRx8o19?^CMf z8ZvrR$L;%+ci(*dOtZ!}deOI8jnFbq`& z(`|-ox{-{UF?8Y(XE#(2Y=Zr;(qX6}m>py&$4#ax4>ptshJ_d^1-6-n%7Zy5R9awJ zC~Hc&sggM^n*dv21}x%;Y$2=~Z>U<>6=tYb7(WqlhZjTXBts3tglR|%ZYG17h6=xh z63-?gjGkwxcDO0tP_ery$YMhk!xCO&kXB);eqLKp2CLHyRf()E_jn3?tEt*E43!Q0 z&P086o2g=RNgr0jdRTd`p}aj9Lbn;pCYBH$c3;I@O+rRldkq0# zd?^{he%K7luQgN~Y`cyYg0gm^r6zaLlmmv!hnxOm zs47_S6B)viA;u2O{*@-LCL^4;h3xaH_1}1R=wH-E&Wh&pp*m2ubA}r;EZW?TyVpF-W zd!|j5Ku4TS)xtU+gBXSb(7e~A{}MYG|+5R4M1#l&6-G!&KOEzD*UwvWrLn#$Jkm z7q(tTM3{fMP4&ZuVw(zkfP!94&0zXdx!v!(|0hmff~UkSPFBWq(HFfuha;pJ;f` zn!#dTK^j{}7d}V9VC(ZVJuG;E7K71T2E0WW+Ax&Ds(m(91d!qd#L-z@*Q~7-sd-60rOW z#t=;DBV$mg}$ePFzjc>&||dhFJ2PB(6b*882+10wZIP83lslf6hBUj9HdL& zu$?cb!<=Bd8i2K?U4=eD#!kD6hDk2oM+@6vq1Z$1ssh%I<>P=bZ-QNQ!PJR%H3%cU z;dT|@K)^{vgsqe9svP!55&>?SVOLpCQotCys)D^U?WzTa&959RXSEu3#xVNWs57mzRX)?-M=P`!|n!sJEN5VpV$n6{Xj!ombHY@{WY zkRgm*YFD|i7#6@*SPbKq*;N@VTw_WsRh#uX+k(~ zkzEbL!i(wie^7832a_-11I=(4Zh~Q#5g*1~X;;35P4tf;1CUf`l}4;jFkdudYGQDaxNu=G9# zDeSqQu7iCKkZ=pr?jgG>fr<6h9JV~pSc1JzP#~E96fL%ouKt_dt8y@O{ll)BVRNP7%TjcK@YRLy)-?B#%`J(dOoH`unKm-Vb~2zKB3QH{HF}Mmsxl|XPUugSPx5j zDL8a|L07?YXud+hzNDtG2&Tf=uk0!V=EH0l>iveAVyOR~K5fGR6=^%{``OM10SU-S zi&9wo8>1Ii4O0+U$!CPaUZZIb*i`~7K1jl_Svgbz%&|FCJ#26|R3EGiaj4kWSyx~R z>^2=L7kW229jXMwFsy+cV;pJ-R)#uMcsn&1>tJ1{sm3`}4s06lP}Q(L%%NJuenIX{o2EHb8SLj9 zCY7+#Lk+_0c!x^p@N(H*=upWW1c2GFVKF6!p-YJXqYtBCZ!^7?J5&)=Ni;1SJkg;# zVS5&e+ILvfH`5Z(b2=4*31`r9Fl`G3gFUbiw&qYV7`c@e_hRV7&N`f%8*r6I>;}s+fBX7d~FSy&G zT4C)Sviadt`hUEd%54rpsW*a}KXc$KZIG!3D$LBtw|r zaA;bF?Yb7C= z^-o^v2g_d}9T@p~kQ#uwNM_Xsv`9ygN`k5H1gQ+z4Rc_7SCA@#Io=N`8HP=t2B|vO z@>!5-hE<;jsdm`>WsvHDy^Fs#Z*Fh=)4*eRWvS9!3Bm{GfU{wbjxOH!Zy^dhj z1=Gx6H3T=gf@w|0(%4{?1!Knrt32rK!BB*uCM;N0!PJStssW}?3Rc}PcXF^AfISf$ zOZbSEnG&qRVd*TsGY$7#A3 zXaQK)LQ}%5eZi^|CcQ{Pu;L|}6lVXEnts6;c_mmSLC34XDh>9$PR6jVgLGlr+rg>~ z=Dx!jBAB%>MAgIUMIoxii@}i)!VM^Wy(C2S!?v$NR9Y1Y_l2lj82NRGGVdhe{t(r6 z4>kEUL}fj|Qv4hKFz5FWRRY6?LsT_vJBa^-WMF3i!cZ4(uy-6Y=pkl9B;EKhEgD1r zM>n$oVTgxCvnWI>HH~GY!c7avjCnqAIIAS{IBzwPVEF0uD}yR-3#%hzqZ`JZ6C+vbEK#H42c$f!8fK&%XfHVL_fRr>{Wr_f)1bRh+)PbS;V%{?~L)p7w z9t=a0BqbKfQsGRx1~x&FEDb@CETzm+nZ-z!B%DZ?DzHahL4k;$0lQ!)^mvrL@ybAr zNgsO}Ezi$utHDr(gNUXENC454N{MBVTw_HL2~`Dlkx+GrJCqDXQWZ9v!FR1?7l~B{ zc9B@sL6KMuz?w44zYa#mD0}=3go6!`{FSW{LkSM;umKLjJ}9EDra83CO+=ij>`CPa z0q4?Hd*~`Sa4S+pSo;@5WH777%1A_Gd2!5=`wiuZSD7Lj%Y`YmbXlCTi->F(d*h?f zi-X9@HZ7#&$jTa_$jU+&sZ7;ksML9k>X(qw!HidsvB3ctJzv>3wIMo#ajzocg4M7d z#v)_uhOJOqN@Q-uq!-#weAofSeF%EP7btuETb4s7?8KhcVW_Od)c9@7z4RSwioNCw zgm(!{J6MoF&AvidxRe(9nyy%;GDT*W2fO>J`4VdS9n%!1e$SYIL$C)n|A7BeWiK9J z{x>h9=?4(%9maI~4;d_H0Yb=^4f~*!G;R>VFyZo{*sEZ4qO!OAObbBAFP3{aEXH2u zw|Bs1>{a`T4|5P2N_-I>R+E1AuMEC^3?e;DKAh=-^so{RK?x{Q#9r(oMT|{ibpOF@ zfFea~hXV)^V^^q55h4~t5h8ZNHW+;vquhq*>XLd99fu4H8|nq)<6g!!xJYhLtI4F_R6?2SO~0`rkZN(Lg8 zY+FNmQ*0_Bg_U$F1zOEg9c5G5(AyAg1#E%kI26wCM~tzlZtNn)tRP~|Y$C2v_Wn5t zjG%}$yKoPihxl_X(|10C9#{<}V-a_z97zko5-8Hn4k*%2Pbv!(tXZqP_6`g^L=^F7 z0Ujd$Y=9#ERO{&qm;pun*#JupL-@B&*+mQ*b`)zrV$d|~A`UIUF5=J**a|CfuT8QF zB;wJqqv=A#qj^xoqisifS|^ zj%BGnp1wSmmO*S9eH^pk1cZjTi}KTqd>7b2h?gjsq1gkg^)48t<$W0(d* zw^3tQ1nZzWk1+v7uA94wu7gdm9tmT zkwg!l&dvx*7ZjI~@#)MOm;tkHV2r^Y*Z@7=8?AsMx-Qznw77|BmP3Z+mV3b-nh<*r z?1#CxTJCKydMiupZI-8upj0XFwdX2!iliI40jRs_dpT$ zCtbib0aief_4h!L^{0C;q~!m!B8t$z2D=FThhPho5{vLZzmN??8-oUl@W19FY7WJ} z0486=;6fSDjk^P7fVhiNpyFZ{qBohA(9vPJi*lf$h-=L|%>UF&n78j*9$~N$yH9IC z0QE04OV~wwu<26z?ma7<2Syjsr0+A&p=cFa@JFklLNDVo{(%*)1eRRJ=!YrAwB(1R zhdlvh!yqi_X8ub_Mf*@r0MS15L(x7YU%^`a34;oX_MsnU_b?_dr>SA`l{D$6{u+J8 z5{3P%4*(_Y2O2t%)Cy@07uv>6T8P#~yE*(v?O3J67;Q3u0p z$m|Ih=d~l6#!vx=p{O~`YiasmyRRm(F#Z~47qv$Y?p=QOZkUaGeTXldqu-)!$ZtXQoWqCQbK(qgDjvWV9N%b=)J(r_1bN*xq+O7l&0#Z2-o zXa0*mC9j;p1M8sZQ_P!*FqfL&L=9mUcAs8F7>8sUO2^49t>{;>ZecB-PeGyRSJZA+ zK9~Ws7Fq=rHA@Y4QL_v{QM05}FbG7?Qc=O)4?Rl{6g^AI9%{N2cPM(69w>Si&#h#D zo}~bao}~?no+a`&mhN?Q5fnX38x%cDTqW@~;Z8+F-BN^I)GdxXSO>BYID0FZMyHbS z?M#c)tdfczriTDNJ&a}-oQHvneZwyJO{N06unF<6unIJ zos6YC#tIa@Og$96%rF$aj5qx+)bKpC1<e-;rWU*CXNIBZXVR-_ z@?Gd>U|gXUPV_W`*t`7p0a!^v<1bE^bR{i#9~%x>h&{W6@URs2!eN+$zbJs3 zpeTUi9$-+HdhxiQJCjld2duxAriVG#k>FnD@%8w_bXWmJVU$P2Uf6;?u8bPhvYm!W zP!vhS57Oi)kg^`4g|y#0=0MRXH9*lQ@my%8 zXp}ObXq0MT2W)?UrmbW&i~sHTKZ@W1%~B04MX?n2AY%j$U>5~bN#~2Lv;sHBqhyX>yl=3)(<$h`oMZ?qwMZ=Wx1S{XezJfa{U=|rgKSIW^AJ#v? z4ym4+{*~$c7%hT5?{V}n4eSA71MW#r&{YlW|0^&^3y6wJJw*mjGS~=M@K**C6qQv9 z?v0J~^^;tj{>E6rUjHJ-Lntb*^uLoq3$p^IzKA~&MfEj|T~uG`PZPckcUbX;RRK|fMK>`` z-=StO`2)*cRAGb9a1ny>&(cLOU-+TSe+=2(qRym-FdvF4tQm?bEVP*d!E7k1uzDz} zuwf{wuX(^`31IKUokDAsKUIR7(&0HWG!3{ zzqd*(DzQ@Rq7v(bkv~)8XBbQ{dLI!{jTJ&sjWxsEVJn`f$UyurbCdX@gKKz=%PVxe&R|COwh4+3 zt`>?8&e6^ShU#q-?DJx%#UMI3#~ajS8j2v;1QTClR8L1+2z@#^+(jqX4Misx`z8w! z%!i_rYlNbc8-}8jOMQza8%n`MH&>2bbaUNMbaQDP%>PQ1a}q$5bk#VBl5QA^k}mme znhf1sB@`W9FBBbJ@;fBF#EK`nx^nE1=;|aL(b;uk7oA=7yA005eeS58P%vTdhx{Cy zR?ubdGXEPeNW``y94fSvzFvv452hv4lrU=*0b%bNy5L{RuGU)Nt6)C%;#5(EvF0Bw zsxD$H8=0cctcSHQg9xJ5?8h!@&BPFc z{Q@n9yQnuyv6sOfNjTps_$C9o=D+Hu>33QY zMK4-NfVu+43+`$cV*%zv>55X=g1yUcAAkecJ1<}~f5f%mLNbDFunCHKwVm{GF0$f_ znl;*KWQv+K7q(yQMIlST@JlH1$1E_g5s%^`Uj#?9MOz0f!e8{UgZSrM#sY%96{=4d zte4YLun9^*MPHllq6MMYJ75L=qQ9-i-E*bo9_Pj2>0x1kIk1fa)j;{?S`W;*3ja^B zU(M+Pm~#z-2Nsu7aF~9rl~B)hwAg3(m-*}tQ4kN0VT|l%p$Vn$Z=+@K&#feVI0X6G zybZV0q&SEMxtf5YL3WH~4BW}|f}%n0grY%C9%p2V2DujI-@|P9oEq(=;Bc^xg2DdB zNw^n2MT@|WCNdO0$Mha=WQxK$ABw`coyQq6MdKVFMxS&u+M#Hi`(gUG^eqtw`~5Zi zj(PnB_ijHhuZ0720c`r2nt#bP<5w$O8?3`#_J?J!f_>OK4$#G4F_;h1H4}_X(PY<7 z@G?5>L24KWQDtwM$UF}TQh5XnbE2x22xEd&WFPY$6?Y^41rvi%5aN%jTbfW*-r;U) zgsM9iipskY=A-g%BAlqc9pUdLdr&w)iSHWGg?hF%Oqux<(}>u*@erV;=~M-c$_ zz#&)}jsLf-h|}pE*_|{!9=UhZ^su0Y z1b*bsXm5}z#6SNb{IPdEf5eb9x! zoI+^Dz4cXEaFCUe(+G)?My8xb;30`jIgJoH#mJP?2>DP>BQ(MkZ)cE7BSJr{fXVNZ zAZ+89f($M>uHcx;g7aaJN{6{01u4(ZY{g&>%>Kj*mkaB#_rgAy{;B0IryE*{m+~2> z7k=TY2pv(Z{hu?~p*N|Q(Tv9cY=Uy^p%xE0_RtUI*h6Bpk(u}n3GQb$eCtct(bZ4O zVITN`g2V718AEW>ApS$N=zhi!?1m*!j!hJkPWA7M8SJf4@vNX-&QWyYQTT^1VOudw z_>~rbIWX~nm0=aE!ro`uZCM9>?y5!wtBl{6j-10NgK{3DbeiFp51h-hIvk&%CFOiZ zH|}yiBX&9qT!^@Pm17fzCL+p#jY2$not!ejE(bTVXBe54V}e!O?_7*Rt%Mvf3wsx= zf^x8gqZ`V(j&Kj_N*Ko-pq%R{g>B)%s)O+HobTZ0@SXGU;vok;Vq;{1;NZqE zqaNnKCdkhw2S9T1uZ<)_+~puhC3eR&62#sI!~THNgH-|SkKw=vl!GFjgsYfsrPBn% z4&cwhk;IvFX+kjH$-p2dM|z>097&wTG&!8}6a*}XQUEzgQjfcwByq%Y_2MK+5&m+b zBwg&BD3Ne-vZNe)@+ywR;2*i#a?j(SiTvciNjv^>;3R%F1y~>KRpl7u#7RF)J33hP z5OFAtCO^nvJBF5kb&#K}308^yIF7PFb}K4D8TQiSsX5HuV1=uK4Pwu*>~fN&eGVCN zk|lI5*On9MT9}&Y4OZ0{~hkj9S%ULNW$OgiUkzV`!r3BVfY!#qw-n$ z7JJw~IC~Rs*yU(Ub-a<8^n9>N!QKlc0pC%X#q5+?g4HmTqcX_}gyX18F_fb+oiMo- zf8xoJndl{S?JM|0IWp4;y@jvQhYJn6oSlhX%4P8lGQ?g7+hG?>US!y#-(+^c99Rz< zU?;45i-Z^B-eDCi7v^HG_S@wwPBH1qS)5iVXK@_MXxVpt;k^!?n^QT<4133WRsyN- z(*oF=U?(i;vfL}6Bhj!Az+{;60fP+=LunDuhZGEZKJ11~A6xG2F!nIR-u*QNfzjVH z*rDeK@n`>^g&}&mVGkX!A`HS!*vo&k?0v8td&z%j5!g6LK@KQZ6PR$4SPLoh1vEHD98|oIjy#LDCfJ0(oSSM*VHR+1s$dO$3&Ynkh&Vo#4dwV$ zEgV`FqH6FjOAJvxF!`_$6_-Mb!93Wz+)6)-4|qrciVqL*qRVCQtl-=gELb0+s$g

0k~%Sj)m7vp}6Vb73!VVU-x=wEQz*1@@}Haa0bLHc{Y> zH1#u@W*9K1V>9`#j-bUM4HuvTg4M7M_9U>M_?Ch#BEL> zG@Y;y%4x((+~qW4H z#}!NS5W|IXOz|8pvm9HL8p`p-#O(;D!Z@Y4jr)L!oIX5{V>__qd~P`0^6UoglQ`tL zgDu!(PPM|a2+nryq+(Fgk>ihv7b7*BYT5gyatgKo61GH9mO~0Gx}A-O$Fj>g$?96f zWUvSJ&*W(1{p^4^E-4AhvB^$Et;2r19G`4J)LJ;3vzyOQLnz@I=Wu|ylihNhWp99+ z-e-d{&&wHW46XAlkJR~`$^3-9ALlRoxlga){G}uy=P*;botAT$wa{^@$BS`Ejqej*8;=t$m7&Gm-A96 z@tfcfOZ)u1ZJfPkt?q;35y^>Bezs_f4mm-ZdZK}DNPpkroMUTPrmuX}xzv`Lq>tU_ zTy8H;`bEDm#kEj3>~os7x+MMVKIc*P-Wz|>iw9k^b;WB=ckJ2QI3E@kIzy$9vWI{7 z(qDYfL=TA)kDqzm{d&ub&L!@O!z_<{!|$>3e!cM(=V4Ll*m>MoIg+dZS%CoV`}OXN zT{HB=mz>kXLswY7)#Lrqf9G2Y6>qy@^}xTKrY?BNnGznCW`(H@r0~-H2Xx76l=bVE zoQH>JA7gp92E6aWJLfIeT)nx~IosWGtmT~(DEWo=>!{bAYa{A$XiK*|l~3|Owa&d? zf7|Lj-kq0Wd6>)n<-+5*e>zVJH?Olik^{M&^1uOI)8(43kCjHwyWVPS#|>6vC*Lr- zvGZ=Q8av@%qZ|98Tz_NpDir16U+K?dq)ortD{Z>g?YQBPy8L6W)MdRp^rl1V@x)&J z`zzF={+2`Pfyd@H=SlALR?CAISNMMBy$|Y@?aoE|$2RBO@bHGEW3mISCZ#hHl%`^AwHAlD>x@1I`@~>K??-_I+H6|W&C;ybi`}Gq?xMuqt z9E^)19Aqv{OvX16hv5V(zet;&KIDqhKX*9i#I8U6kU17P-3nDsEcuz&JfzpW%^b_| zdvsgwjM9hn;$NJH>7L_Uj!2ngl-O6(lt2jSuY}iJGp8vf#UA7zJ;Mr<9|$w+A?=k^ za;|eZB2p=4l0QU6AjBsR>MfVKVvTft$&0RO`nl^|A!BO^nCA~zuAT2VJ^G`UT~qZn zFS;h^;n$qJW{io|>km;MNb2wh^;^Fsc+&Y2O-gcypoLc|5qIQju;*9d*fe&@spQIYk`wn9i7`_do;X-w8{?5Cs^ zyfr2QrG-kJb4W?&`b%1kXOZ8tDUingTK&mV*DO8#U(V?fgM{evhwuc-@lvhcGQ<*v zR8Q5+wbE`0M8C`D8Kv*}!Z}`V*x{O}pIG7wo=niPI4fvTAm~}O(&$t5={K_8R%E#x zzEydcpr!MypecbYBWk7fr|9VSoSsN-(p7o96{6m^m6878n>nnybM$5J(EuLX7)L}c z;d+)?;d-cu{LICq>1p1lQF1o8YuAnxNu6rz574a4G3lxctCK+y7^+3>}l= zniO0>wTw-AV5}=VvX>~?8>}b|fu-pyf6}%gLmREma9;yQCiCWy#%wESTcCmO;))YC z;F_tA`ZoheUacFu)e6Ef5#P^ziec(&rhcoLbi==$j>ukuCg)l~&A__<4^j(UaFcb$ zu+wx768<1Ph@0W zf9QT|LNEHzIn$kdp_RqR^w4ZoE_WRkCChX(|6G57kxkB7zD>^TXy5Q9K=j4_6aus6 z6xnLL$ZjR&b7y!&I?kz=SV5%T@-r9TZ!K$Qb~_ij8;UI7R?AbFQ9j@GI-{4RB$V;f zaj6wT$|OJY{rmLpD_ygFg-j%|v{^6yK~XEZ>ph2jVXJ7~k$IE#Ll8aW{YLT;(n*u{^BP z{uGKtAN|;Qg3Y{IANbh0)ER!Y6(CW%mlwZ&Lia|N{n@>y&-S|ES{p1^&`o9fn@`x4 z=H0-Rbi8Z!1gTpw9(6ZZ=~PNOQ}pxixW?+;J){&J8(|n3{2qTtQ|>OO0yz$N5{P zRHE@S-S-^O9bdcR@A%wlhUfZyivlhFIUkq`jJQa5#Why8RaRVO{^M@F=2XVx?$4d` z+`ZRXzIB1fm*1^7Ud5&0o6nuo-N|K#_+Ib#J)#%iMt?(>1ya6^l&_SOd!0`Au-`W; z;JcH!mvc>fv)Acz7vE^5JmS03?>p`be7pU=Ba<}bZn@g|FK8!y;f!*(++-!t9Z3Ge z>X9H*-J8lS-{FAoKdSZXUpN=Kllraor7qy{P_=%7&zS0jFP)~l=1|{js`cVr#>krK zu1R;i&iu=DD3x}LmH0^MPN}x8)_?!fx!l&cTYvwh^C(+$rC$CO_Ws-Tj<1{>Y#Dd! z=f84pjXh@FAuCeu-b2dz{$KpdK_BaB*+NK}>nqgkpW`)hw_qko}JHBy-JL~*Toq9=_D_k%6#_4vZziI_5 z)NhZ+ZL7Fd_}$8M%%{#t`qpop6K#F3O0tW5TU?nWrS1CTZ^*9fYrW`O>RI%)K1W#d zroQJ}=Ze@ps}C9fbv0H-p@E^cSnDQT@xB0uO$`0Y>a?Cfr+urB z9bi-WU_ZBqiT7KvVgpy;7x*An%I6HU-}{|0?z%@U?|iPP@-y`vdhvJM_B1|ac}S1? zJWjbo@BWS(rIx2HkIsO{(LRq^+V#Ej2)BC0@*UZe&b`Cxt;-IwC+($S+Wa9}t#o$&eLsWy?VzC*J^wAm)u{>a2=*SKeA$V1Z+8COa034SA)&g^p$?(N9S_i zYAN^I$$eJREqX$zD~5MD4}9+oAEOeiz>*8Pp%A&mZ3-y;m~*KZ1w`t^aIoU3h({d)Bv_Vn-c4j=c+J=X?X*l+s#LFd}o zAGxja2`B=kt^dNxwk*(!D=TE)uFxe3F4NufrR5tP@O4*c?|j$7kxPH$S5|+D<&hi+!oqA}u+)k}>$NN1-Zhg+$ExYjb?h?O8X&{e? z-Fo*l`Zaq$%hQnGcjRW|t6QwS3pXQ^-96u1xsBY2;2XFRnd?sd!SZbir2NV)qi!+c z*%j9MLwEuWa2xT5C{+G{6}2T0b-T}F^zFs&e>sC9x=EzdA2vUb$g*4Ho@>3k??)?< zkt+Xwv);16wKk%hY1aIoLp-0jS-&8$!hW(mM(T9d&2ou3%y(aF$i-Lp*LO z*IGQXjUnGgw!?~mG`w8z-b{g#hn>;3Hbb8_>^#jj^l$x_BG-J~Gwckt_1X0I!;I57 zyPp0BY_sbP!ZL@xQrI7?9~X9o=#Pb+rXF{Iahx_*pCJsNpdS~8P1FYtu&{nT-#@N% zDT9Z9w%sbz$dWYvhLI&{xjWWj`Hn0{FO`Y!df&3!NA&(se}F&}dCT-C(_B%vuXas! zw@4zO`#5Uk;ub4cfU$~IX)*#u3O~)d##ukIb>Za@04A)Wi&KX1c+fV$v`1#Mx z^;bEK2p<0Boz@jWHiGgq&$`aKNz_}LU6b|hT-Ssd;w~FBnSJI&t8fxue&%u4=_j&W z^CDz8hWR}Qd|r4)TxX4-`F7VFcZ1(| zO%b!gt?Wm3czdt4c6jMz*-e132rIzI9!_8T=N>N1cLy6!_asGH!Eyp6-FdBk;xa~` zPd-yjh%|qQ);s)6uzS46Cu17?3(V*m~MIZla4R$%2NG%u$i!I;bK&NJ2t)FnYPPHX1)2=b3Tb`(o9phSV>q^v* zgu0Fhk<}^eup#~XU9MGkpa8NZ9;SCpc5N6V`4y0gdANUi)rfD+;Sz1N&0M3u33Z*I zqh`6BQGu(bBf-jiWOh8|6~Vz;cS)kH93jPI>^O30 z}`ZR__1@gE(vqFV{e|~?*kc^9{x$I ztb%1>mY?~4@yHEJu=YmOL`qGm%rMoiVC#iQSj1$T~`&=f>V~l>Up`wIcTT$^Nzr z43gp$E8UUZ#c`L7>@FtjgmBqk`Gb@NrcTsl`nYhWa@vtr+>t8%bg4dgGK*SIxGUP$ zyj~v&cb#FYdr_Y;$#sr%XuTCWMQ?qJTY&E-xt2MHj`F84dSgA=<*{X@$*@1!me;0V zp6og`){s|)h|Ws-*u%dl&5BtXDAdkN^t~%wae8}%>oC5hd;VnCMm-S0aG5sokf;gA zSy7tXXh;*6ZoqpfcKtCOMv2XR8 zt-t%p72#WQqw#7v-b$n|5N!O#Rw<90;#%RXeAT)ZriwTtLf1}lMcQg!)z43HEqB)Z zw_o^F{JJ*iMN_47H|jHlSsV2&!upN+`KhiGodX-K+%}2)BvLPlLQtzR^vO}K6J}q^ zJ6n7Wm_vO${Hrqjtq@p;FTW5q$(QK5zKL=j=BtxH2EjA>GPsltcFjr zrmBpYY}zGzn#0_!evgs<*s^Qn4u6_E_7y8yLm=9+U3%wq*Ftxt-(w`&_+274 zNQzxC#$TjtqIme{oMaU#o6^hAe5*i@nZW{<|BCNc&1RmgALo`$?A%!$Ys=a!%`ivj zcwC{j%FTL*hrUS9*7tZ^>+D0>Lwfszt`#9NUIh5>CZlJ zAqhXj%04-e{Utl}?iklGwz8Kc`=vT`CJTQ1%aZpxT{FP_Vat|Lc{JsW%40%2xmNKn z+-jwouHT-BY{;BN)00RUFFG-_N4^ z)iymR)^+Oa`-1%GNGD1<1ov&RYXa%)-5%Hy#JY}kn&(;Gx%$ZgXS7~1+qKvhd7eIR zw(DwJQiuM2w(F|cC++?;)BI_+o^M6U!7e}Zm2GliU*emml5v+mz_8Do0M~ERpUh#! z?0nH05+mw}%xwpBLm#T+jdNYI-D&>uRKVsye&(8O`ue%9E8KDIR=lA=N4%S-BjOkb z$pxeOyP`lp9!GyC?b06$V=vU>=864ny?P$2o_bf`Blgll{pLIt-YLpovB1Jxw%w{& zSs)$zH(c@#s1F74b;(w?m%guFOl$5rx_+ zch3$hsrrD&%5&uYZM{2j=OG?HoNe9MIapK64)xr7wjLi(tkeQ4R!1PYQ_hzAs1>>< zo@-#M-?uN|8%3HT|2wu2T~>1pw;m1bQOvijS`_IgpGQa+v(V*sw(qhMZqjQXb}iQV z3mNVMZ|RE{x;EIm-u^?skm%xbQwv=ww%m90%tfvvY}vc?8H*T@IlJ^7!tQtUn~Pj) zXHWQ@O_W?L0&N}XPdYqsv3TlCv@G3dSx#Q;njYSDft53_An^T6@0kbgkTV8PEOxE1 z#}@e~L0AHl=Z{|xNxP@eiagSf?`|E{kKq?vz9aqk=vGTMkw{K*0qXMy*kom`%$v6A z&l4D*-IrJ~N4o9Ytyb@()AI>Mhj>Q$J!|nSJ=F899Jwc78!?P$m)~=w`>x6H-3I&a zR73@lb*Yu~Kwy9!oudyfW1n%uQrFVhv-clT{qD=G;3L)leT!6otvjdqkRraj#VUV0 zHEr~JHU?^aJ)Sa5zTZx#_!h|ZzxZ=XA&iHA`W04!ae;!(+9Cy8;!e2I^5_V7glzF| z1rl8+*gC!Xc^Et4XMcK0{`A7URGAZ#UDIv7*Xz}Xv%*v(gA~@5>3a_6%3XJZ6*gYP z!tOhgTr*;g|M*K5SXD!>vBHcD%cnN`hNUh@a)r8wuC;tehTiVYdUq0Yztmrhk->8$ z9wI}DUcuD(ao~_#ips2XhG=p5ncto!m$WFoV+A*KvHo(7-0;3|nsnP%Tg{F7N|8Ne ztYo!FID#ur>rHy~5i-GjT;z%;u~z;ayY=HoQ1!Sw^v6fATdc1hi5YE6+^cu2bghlu z_5C5mO7~|>clmzi>sfkt1=^3tu5*p`ZQb&5@4C?{eBJ+H?_I#Es=EHs#YKg^Q7K4K zNw$iLiinDeii(PbcQjH|DoRo-Qc~X{^KDb{rWqyXWs*{&*_(N#qBNtVq$0JnqB7H@ z#7l`uc|*iJzp=)cTUl5gr-{lr0M>(E zU?YLLxon)M_6?x)yvPh!Uz$EqbsI=4ZDBaHM#qV2@j#4>g^O7O+pK2rX*yBb@#%rK zj)A#3%udqmpC$-DhRb{=yQsWY@xn}@;~WWUC7Cu;7LMD^29RAT1q%y2EfMkcEMabPTalNmHvQ}F4N zs8+?HwdG4TGMH}oge0;9_Z?>I;u(#MxJo4P9rfTalt@V-HUcBmek!HXB6V;Wa=O1r z*>1zctQ|Ffm~N{U&?Tc_ip z1T}V~ZFoT1$IPJZN(w%G61Z?aft`T(UChlidEkys;8u0hDC~|b6t2rTbxGiYnLNrC z5fD?tx&>PI>{xmh(6>32#W`Wp&_uy!*I2dlacuC`f>|p}Lnn6eSatPr+c1?r8XFUN zpR!$7pPL9(S;%(_s1~kEH()G#-G*aqodfbeV_jD-IWva#jQTcD#p0z@m@X&$!5DRh znE2?0Q7g<%I!(`yQG@1V5HwGWdBZE_m9p)8GegLd6Y4IBL`YPMge!uv62Uq!9bd5e zLaiP$hED4BHII`e2{T;Ber7b+lDVnv|ESHUc^saJMD1o9t_t~KG>sO+R?tzDbr3iy z0aNw!`sV?c;QTa6I`SM7zUElIY{&9cCY zvnrobed4V*%6NT|c z^tos5u(c1YJ;Cf)y_(%f>|%^O2k$`Yj7nx3y1Q%}t?Y@m!JcV`cC) z*5{JiIt1*P37B$y{jJ4X;2BO8`A`Zz?~LS$T|SO-1yl&v)#w+GR68eNmn67~)n;h5 zsUvA8Z$LoQpUl{=%V^|CHFhFR1urloTW5dzNcF}+aRfES0Rc%?qti#wACUY-a z)TY87sT~joZHZCvX+E5%z~iStEw`4{_A}G)IU7&&X-xL6Fr!kJ?zVV!1QYGRqN~iv z(2T|Lc-&M$n;OX&C#pRznL9Aq+TdgH?4@5x#@L_bsdgu0R)ph~e&FUte={8Hzq)j?~#_LCqPLOb(W6$ zj@#I0rh||zOjp&l12YLOtr*OFEAz-O)5)m(d*I{?g-enYd{zx}J9(z28|=v$aJ9Xu z!?^8rPC?r%_hN3mu8W?-c#!Jwi0!5oXKdFzYa5^u5}RC7gZkjHY!;AYMzCr3Hcq`2f>Hv?&eL#Qhj;rnRx_vK zL2fBXPFiHWG8L6v+eGb{iZ${rpGl?0B3=i6(QojIw-A3*@M$`P=H}g7tQUrB7V*o=Sx2>OH##a_^z%8`Z*e%{Z10yl%!(28qO**>=88*8-@! zW*ngy!PbdHv6h%P&^65X^VDT7!~z4@kSiTsBpoGS#tYLmF}XZ|CMNO_JJ1_X#6==k zm)@QMs@tQs;mzlQ8F&lZ_SZ}WCY{Kyc@Vv`Ot`LUnQn6NLICC=xy@MB#q4M@dzt+) z6#~^F;A*HRVz}tBq0~7bu{o>9ArlmQR>$yJ$q{Loo<_H1ZV(F6%yq5NulDEsu9#`-9GKshxremnef?$X^4Co4o0hj%5wmPKrxK=~ zoQ0lL5(W>0cU)BEESry4YA3;h(`o0a3!94MF=2SRt&4APIMZs?lBY4hS($EY>FbDK zTD}^*3O)6!GT)nQsVL2pNW%U|^>;d+06KcIOuA}Qgk9f%Jz{I_nbcFg`UsYp!{M{d zJ=-`aIq*vh6=9u+UD0awQ<&s6n~l>g1s#~10Z9rzTcfWX+<++~t0N1z3ipXlPYv?XO?me36DkddFekWk~GO@l5QB&qgAUpwnse6yQ>%HU;wS?u0A4|-b4K{ z2M_IHquE?F7Yz2i+lRjOu4IhnGYcNB)87M?n-v zB?AKXkKs6~bmZUmREy^$A1MjU$k)l;*^`DAG!kKyXvSig`X@93$FYVh@{v8&>VHBb zUKp+-X$4cn$P|Xly`SnqLo&)r7&Kp^;PXKbI)T(R;E*s}eMI$8`<_I5iyY69yDDmA z51MqeSEHQh9?`@BhN=|OKh4GSzyVn&K)4DEp((H}9R z>Ya(D;DKaj`)R8SwAD;(?WNtr+$`bV+r) zgtOLLo`#be{f5O!)ru|GL}yBObrq){%(HoMh?yK!+BCN766(f6D*YLhb?S8HuG7hB z)SXY3YHX-oh4Jt9$?{QT~8l$Q)B;S>l+v@F}S+q#%_3Ts>Bv%o-&LRZWYCc z;Ip8ctSab~p-!Evb54O#ym&+>`+NfY9_A?5v#R0Ku^YMUVXBz8iNY<@MRX#9S0pvh z;>=&^Ox7*Yxv_1`17Z=)E1dN)) z+&Ik*h~P!bv{A1J9-^*2A}+oe^N`RdF%P-o+ewu_XY=;*cV1Fg|Mraz z^~jf6O@O`wzu3pvo~r`x=*oQ>FGh9@NVRh9UCn(ywSN&tz--}W!43tVAHwNrOJC)9 zo(^;g*H7o;jd0f{n#v@JvEt<{G{4Mbx*$ z$!ad=zNfQV{37-;77C+OtF7*gS?VrZxcMZZ)*I($o?=zk^vu~=g)D{-Joz*;PHNSW zoz<+xw)oHoR$5v|>N704P}kwrPW*V@Jld9*#a!0}{fADfTMovsDq*-PXF(@=M%Xo= z_+PBy8hD=Rr1s^&n+HD23|D3M>EwE@(J?SlxUMxuqfURT^uuuWOPAo0bKpW&j?o5e`#elFT8f9RemUyFrM9-7$xGCWOEIMl`ltG6DV?!cqH306Y8(9$EBdQ8UiOYE zWf>k7`7c!uE(0%hIrD;*$1*%P`*0Z!_Uv7u{#b@Rr5P_Xubkd{a;UqO+j@Fdu2P*> z*oHJrK_U`fRR@>b#&~R3OP1S)Hmn9I_%$NkrgHcB2K&}N!85%Xd{X563R{3@Ih)}sM!9JmCr?TDamHZ|=%30sO4U-^s@=bML zF0#D|8#H%emRXpKeb>OP7$Z+%hoLgp7U<=tvEY}1c@rjY8hSUTa?an%sR`$|ATwXa ziMhCU)ykJ`y}izDH%HqZ>hLSJ09yqi!w~DsxKIBN{jmR~tF#h@PR6AB-TQ6*DE~ z;&t0hPe(tsgJAIh)%o9OGYNy$gMeP~Lu&ZobD-d6@^JP)-JJSJ-S5BO4?iP z*RaYTr`M8H&3f9|o}~J{379=e%>(rEpIpQHtyOQLlCvhO8sd6SQT;Z6yKstd3pRjT zJ4NjvYG$%fBR7J2E?M>42%U_()I311>bp#C--fui(Ack3%|;Yc)ZMDbCY(nxQq`hO zz+=+XC!1_zY*jNkZ*!ZfpU-0psrzPh*YGFRh|RDV_KaGz*>;;}#zOTC!L;X9%Pml@ z+M>p90Y7wydT9#^YF&}qu?1)1%1X_0zzeWjk*=yz)o&|mAZw4>u$83C)bCr7gt#Bn zkOJWT2i4OBwrQU0%GK`$pl?5_hP(w>@uyn(7WQWH&vMe%$%(wTkutw?s^8nt&3;ME z1N5r6R72a$=FYN@XXPdJJ8^4kROff#AH4srQB&Trb@fa()S`F5$hE26@8AqUt?jbe zoD%TZVJ{w~M;GFNw|8STxe(5&;$~)MDUUqNVDm3w8$RV-n|I>_=we|3%)}e6p7v1H z_+48oujIhX+@wYW^UMoW559{=QPAufwRyvn_%h0jAoc1WXk0DdLpwbXr24&wGA+AR zJ^dcaG}fVZyobhSwBrau)lL3zjW}J{Q1#emYvx(mPL0?G!E+td)7wzk-kmSYP_WGw zXshkSYQbp0u3jCv9ob6js>W}JUrX((o(A--j9|%8$^)hqy!$?KUln;-?Tf&sTNSB=byv0> zP>boK`t6{K>Z|4v?$=*!AULYOI!N^L{;K5%$W880HU0ynYMq!5SL1%cf#kdoY=NFx z!_?cvD;UP9i&uZ{^X;hoi_o|>-KJU=LBDK-8ear^86(w8MYg+rgGRBexB9cd)=9Pf z5Y4k>j2iJFDmH#R^P*fu+d#ebAzD@GRJHpt(Jl%fQ-AHW z4fI`@!MwfdDPNS@gkl>G(jQ=2;`Q$BkFz-9xV;$9RPTI1W)P#`&>i^s%VaY_Pa%6= zR%ePaIg6G_Vy0$XMF030j1rk7t}}-OEGxFvsKp;+%9}0IK-XN@S#w40AB5*H|9;=r zv^B(OZa@(wFJSMSu1!p=Q6ZmT0$M1H3eD(KgLx(*jh8uOqGmLLXCjScVeHq8Z!f7c zpJ0-^R~Rl+i!Z6MyTC|zjw7d;5d~~=sja(UYKAae%gT;0wF?=m6h;Os7(SP)k+BPy zJDH!SNiV_EXFr+Cx)QjxS}iVt)huDSl%J?p`$~|dO~NSAR`K#v$fro4|MQ%{bDDAX zqH+>roiM63W9vnAgcvb0UryGHhw$O2&k(t{OqE>|`rBdZGh}IoFfz49tBY4u>?B+C ziqCD{fl)77>gUvjD=Og$TeM0bcD2m2=W69Qz)qP5c0SlQ&&XlzGR>Z*R&^!o)OmGJDGIAv7_Pe6c3y2QMMmZ;*2vb8Ej@ol zIbv|&5l0OC&F@U9$S;uS%v=_9t*XXCF!BNxtapE5YZe%_lGz8eok3uyRAcvS?HBYg zJCE5Zntkz{I`Rb)w*3`mq-n-G=Tx`dVEBw=FBjv-9G_X|lyf&$DWfJb1E2dfgU=o3 z)DOE+i>Y@pBd?_}LQIX0>h3+*hbwxMBXWI0?DlhN@gD5orEX>hovxwabM>sWPLI9V zp<7?b>{#88KR>Iw?X}$*kohh%QZ-}FS#@=&7G{XayfYwUo5kFoztp}j zp=IE+JPF$^%*RaTodIF*TNv$3#%S|E>V9aZ{>TDRm=2qn*!QPe{S^u&`WI&4bCAq< z=1+B%7$;9MgFd!I!Kcrk>YjaIl*v4@T$k^uDz%dsi5FOdKk^HW)m18FKNx#!Es@`W z4>=KIT@cS3!t~0uQI$HfA9XmlrG@eB8Db1>UX1x!Tre|T1-tr;TKqM%5^iONYn?ph zj5_lzhViD~U>82JJ#)kL!-VE%)a~Ek-O>G>nc;GAKm8#s7ax)75SY+~*|}T}hR;T@ zQ|Ng?V;uJh%oTRAW~aB{$bDPT-r z7Eag8&9kQ|1K3A@kQrIDbU^TV?X((OhAfwEU`By%ocm6zH_DKyEo6wfZ<#{2hO= z-5NOKAakR2bgL?_sNg+#BWm6cwtj(GhnYP}XJsVVd2!gCKJ|mGO+d!a%#PIs;Z>W%IuYin; z7RK`?qiZV~c4J#%d&P&RKkKyFhkjE{55Y@r_hp7_tE}g5W-YW;i}0#Uz`_UF_m=1+ zUp%RHLN6e*3G2CBz&j^Z^TXr<{FxD@V|whQnspc>Qp!!tNY!OJ^dx#zV?0m!{xJOb z+(2gMX}y4xs?pCdSSv{=)JZsULf!MTtyhZ~4{_QmG~QrTiCkG z+s`?n&cH}OY7jGuw2`4F)T|@6HqG;)b<)agWis2hOl!ef#X7NPk8{+CYVZ+T0L|&5 z@bkx$lXERu)K&679al$=pev^YGb2lv#|Ou0And3HmZLcQLzo++V_1A#%_>KN%Y{*( z6Fdd>j-myAS&lQg!EIR0RbO3>tB?x3tm5~J&8KBXTjt^-Iy3k*m1L)=r%vESoZ-LN z>;VbkEaZBy)$}+GIbmw?FGzKgtYhNwDoB_BQ#(oLaLS#{!oBij&sq16aoOEcE$foVCNh+L{M?KVYfu=Mh0> zXIl#@ZEEM~ZLtS1kWJ9}7z#dfj;TRM(TcnWGh;i`4WB!X(Iggc_8!CGSpRnTz@|Fz5hOla&u8Ky-s7$*CB+9A@HzS(C=S+pVrvjcREuJ-855`}sP&AOk4gvZei`q2K6e7GDN)vGIl7FtA!GE-$uCc!#|n`CU959{RoyCWqdm(%R!>%9 z6i)GtAAp6-2X8%(lx{DXLHaBQ49XG-5QWw!t7kl{`Roi2EBm2 zrOa?>#`A|&)4$MU)_uW@Je|xt4y%cOA!EjFX3WqUA%~Z5wYO4l|Al%<+@pT|%htmy z;j7E$;|rB=7LP}>fSGS<&p(SpTjBfIK#CgdXK$hYjlB)e#C__cv#969{i?}1G^xO^ zRm3^ii~L&MeGc^&U#9+jj`l2%sK3u)L-4?_Y{Q!#>fEB9K5uL3dFZHm^*owi>@nu$ zs$a73Y~sRs*sndNnp{A8Nj$E`T!2i%Nu37B1gY0AVCyI2r26Or(h&HYYH|^<;5T(Q z!MNYmmWzn9>=f$+(ryjj5U<8zv)oFiuVQzcx^88N+He&gvpL;Ky67k!Uh*+9%lN4`78fFWx&}<7w~cnIEfmdqOB~h-&F&@9mp8 zl!Y?XBLQ%U4|~}IJWGeFr@ib0ZR>|I52xT?Y=KEqxtBf2vm##AkmB~?%uG@*)L?#d zueUwKt9WEB?=;_{XmY9xylswFmid-b=sy{&KJXr`5@?jNBu3~69*=2<>k zO>SWCZ3`R2ym$|FcPq?)-fv(J@eCWQ4iYneY+YvGhKMvjq1N?u!&_xEv^WA%ImG6sE z%T2{sk?eS6aX-F_gyKJRzZzi!UU|P-WCLD1O&uhdHA8i_+k1Pio1yNr+wrjBA+^H} zxlw7VhG2D?>gNZ3ZMvH6XCGrro5u+)^5LAG_Crn&JgshSjC2Gos-;JkZB$lc)I`xD z7A;hdn9rOWqoz|{P~S9$x%4H<)&%B*m#Gm=>=B-^%hf!BIm^`sg5}HAL4uJhl&vXX z<_gu1;QkfrL4ppYRyKuxkx~Z9S)mpRTDYO}YfyZ!AgJ)>8udH#@zTctJ- z%*ay*{h=H7ifVb2eNyPd81K!qhhgXx4*cSW@>mpuzbW|K^U*eSlYMxQlkVfXK7=0U z4V^hc91n;c1|wJK4+*_7=2I=)&>MnI`(##mIsxKF-f7$}j3+~QMjt2<>Mf7Fg+nL3g;J_X2&&4RqW+*j*A}H54xNDoKdU1dZm7x(T)F-2y$DZ&~R9?h=6b zai6PU+~!81v>tjP=n;_jKEYL$g1;$j2c3%EN+X3StMp~;faH6qtmuayR9_=??8$4tc~w4v1g^z;l5kxBt!T~R2IS`=KqG#9FQV?Rp{CgeP}A& z*xhJROvRjgT-C(cBicn{mPx_(8{2)r#MoDm-Gny2OF#Cx)%m~K0lNI(qg=C2Lzm?imY6o~y5~3Yo)3#Khc(um-|HhL4A2~qSsN?^rL+S+l?{I)t|7SXc z|BVjNEW@c%lJI{{v$t4#(f>LJ$T45@f4B^bf=lfNwD_}F4{ZEi?*NlT zp*{!ri16VB%wR1!?EqgEx%wQS#ZLVW@B=YKK7+!~g6rO3O~C(gfLu;SJu^U;1H8MP z=~k{ez}ZYN{$F>1`z0aT0bUebSGjej_kSFqb|3$Z8KBuZwEX}79iTmw2haK(U>Ht> z)^UKbqEMd$oFx4J)!DHbqDA<_czrt!5+~+_Z*jL zeKXEKWF{$}{4hqKDbrH9v^~!xt^9w3M+?LK?YRkr;BN|N1lKilQwQcdOs-*A?VJFe ze4>?prY_xS=!9JXgNAjh0zDycM}G-)YM)kl+GVB;luH728@?hqPp}Q+s)=JJrE_RC z^ePNd4*o5Qy6ITuc0!JBm9o;$gYIxczW}-`;WU%eC2$v#MoVU^A)>pBU<~MyZsdvX zPQDfqQ-8SLKE-d)L(Dc72GM(h-q{VE&MMIJ#On=_d_S@CNJnlPv>KwY+JaxFXU1>b z3`MGjDiOTJRJ$>Pqmb(g-gO2{NiQ$1^9ErYaeL9X)_G^)>oIa%n3EN{)f3@wh>&(G ztP%bR`b}==XF#WRVwI;?)u}B-OE0=j;AE%#F+<>E0vFzg%0jp#-FKde83rvRX>-6G zovOr2-v)U~kd;0J38IauaaP7KFnYOB7y&xfr#pFSJ5imuB=1ze++z=Mriojc`#Vq7 zGay1C2|p4%^eE>qjED#o_6dJv|QMob`#GRdVaXWw$Fsla}rV6J~o&yhW77eJVW%f9ZioHy8mM=7jH&?EdzVSZhia$<~t277&L)% zP$-Jo5`mAvNP(jShDpRJ^+a4BUm})x(eH9B5vTr#h{Llu$IG5!m@Rnevy873yi#D6 zz;z;@B`}xh4y1nF0#=Os7sG6U6(pb<53{%GbV$w?&8Hm~1l>$D+_VB6j)MAlx^MFa zdDN;*O=n(b=A4!W_$bR6;gF!7>yA#thLzr}9(kg>lkdTF=M9EFg>ky=*BZgjdgyBOIu4pWx(PC|LC< zkrY=QI^s8c>M1&+yDK_;cHC{%5Z#?TZGgC|N}@a6DbUe8cXWI~-+XPUP7aWPTk4^= ztjA8U&{I#zmhm(gp+H@mifEG*B~8F6K+uF*4tG(Ra#s2bNYOB4r9S`~Wyng;qxyHJ z@QNsebm7*aN9aA0kVn9$a0)--Ws97g%rw>u?f{R%fJ)B5LyQ`C0ye|1)G2~`Bz{u_ z?-RkH$SIBT9^ssj5TSP!oTeQVs-(pH88uRbPmxe~RPb;{jkSV1X>g-ZCIs5br*K?w zhlv?R!>*hFZNQq)<{0-#qKC+7yU+aBdQRinF z5-$2Q^P^yuI|MmjV<_H}5Wz5(Ggb)RL2#?$0cpKOBG=j+$vcPc1|1>cbB2DWxyt8? znePEyYZmIKa%aq;cDX+!L_|+7QDX2*gKMrPXvA{ zFk1BWlh3E5j349V_cxx$AJl>`2_u~u2JU+?A6P#42bZPpzY$13xyGDGb?_d0hX&it zN}*!4Nj2&x3GtqsRo%gKokRvOeLra;q*pWiS&}-zq+%i`F+V8yLWw!5I~SSu9NN=Z zPxTNwfb#*aP~fIq&NRCTOP*vrb1K780yTZU;86k{ zrq(sLT)u?$>f@&izlRp|na&Z<6@$wJ>WGs>Vg6Lrc{;jgtkZ=3@@8;Ev66vIv9nh) zu-;-v)5#9X*E8EwotkC8)i*;5LRJIDgD}fICzveopXu59PiG?ii7;0d*^f&BoPtZD zw%oTjw`iNthYC)WMj=OV?K>w5&ZU5vv*6TVC=5GWrvbDzSMxM|-YSARqW1;&T>49c z=AAwfOpiXF3yvmghEk2219pR4Dm62&t_&U(iK@Pwq0@rb3cR@w)7uH`A<*c@^fhYLRcg4ZGm42JRtCQftLjOM03Qg1o}%t90I!uy|2JXp^p@N zvI#K>o-Tw;fw3aELhwxj-x1jQ99Pj~DO=o-Wi|yXot6xgMDJ(02{}ZbwXyq1O`T@L z^$FZ(h{4bWY`9c#-FM2LtI$v1dkFJUr)sK@pZ%!uhZ~WHJIsO!sd zpu8QFE^-9|Gp2J@)CwMVKjS%q7fH!w3cgNY9?6YHE@Gq&R7wO|F-`CkQOFgTF^wZA z68u_(bsclkb1Wy0xtvh;#+Cna{prRrKQ6t|>X^?8T~C8^kan7UMfPX6q36-5=ea-1 zJlZvlJ|uMOba~7LQ@)Eix1m}cj~^}8V`b#B&Y4n{m>Knzfutx#{W=Tl1@FC_-_(~F zhKo9xq(GkgBmCDdhSU47Ni~c&1gDiTg~)Vnkt9!{{1e7wg>IGK#ro=My8YG(eX;uK zFn}Y_b2h7jHG=B@7=fxuw|5x7?0;z2f7gFXIN5FMOh%eL2ma5}9QGe=vpqeH(uJ8W z@h}(P%Nyv6K8T?AB63AQD}2JO!~DP>$*ad zqTC=HM~0dJ<__XA(HUPQI8BHtJS=)s7&UqWQlVMt)D5wE@G`!rRvYHnBkE0vj!5ho zg%Tkr(=di!;)JG&z(Y`=rH7SHJJCch5P6-zT+omu=M9Y2U{J2C3R_v>2KsWL`^Rz# zQDhXX`o}<{PTW!C29OYK%B8-=dD2rx&KEv?`!>^qw=%2{m?{ST({g)<$o-egZLA+$ z%Wc0JRr8#^waWa5-M@i9#m)}dn8Z&KnpbkZ{^HS~!Fg!WD-r)-H3#r-Mgf8oCN?qy-mq!magLD)h<^nP2!3!%xL7 z*C!f;Wk1BA5tv$JtD$vELbo)lyxu;Etw+9460Y}Ww0ttjBYtxQ3SflPat^g%Ac}z+ zOz~&#b7*|W^hKg~KN&#?1-TDkz9dMO;9g`P^wu!lRn_p_xmff)jQ1cu9K zI#+NF{RP+hRv09Fhrm#=6DoKlLuxl14#aVp@6X`TYQh!O<8gb$h~P)DAVf%%7iO)M z9W6x>{S9Zt_?F?1Vo!J8kFKZMOJ`1+(dlol z9__0`uo`Ru-U6ZP9UmcBD$vMc`A=-$k_V?TY5|&b{Q@q2J;n%@t`&i*hxX<{n%uIyJKp1G!G$+nL^0Vzm%cXq25%BB#5ORqkcTP(+W49Q{qfD)#|sWY3`2hp5G*n1A5; zg}_jORz*(_E`KW>w+);1GEYiq1{p`lz>i$7l}-tD(aCpfg(Cb-g=tlwu0r%S@*x5) za2)CpoZZ@udZqQ)I|@1tP81Kt?@9ojzHp`NI{Mr7(BG+t{#iZruWqE9@f(N0aF_GL z_0TWZL#L-8?(95X5B*6`#7_#BhjXFoCEHO;YxjkprjohMljZbAGK~q%+|cgTJ#h)vN9!t zex{mXd@MyjUGP~*AeDxd-USKl1o{rqr_!TfmFq4U>Ey*_NREnv13$P~r?FfZ_YxJM z3nC=PVWm%oAE6Sl(x-zy$PIl4=&_(*6#IIl+yxpXh3%kW=!l(AAjfZw;6)S|4SQDl z>!4E!Sm_%;Zwva}-kjqn@HYh;GDaDwL#KvGM>c52T89DL4|JN3Tj_La#vMHt^l&%w zV?d`8wc2S8x|14~mC+Fl@?2JW80a+2S?R|>r~0?jy-@e)(KpZ=U88fy)J60d81%fr zYN!Ns8fdKa#h^#Jp;v%T3lpon4OLvX0v2)?R^UIsd4*qJwyTd7`oIMR?J1nUIl+`~+F zOl5dd;E$qDE}FtYY4v(#q?c&J8gM)_S@h3Bka_Ybc38KJ$4c{-2h?7RK z!9l`*UGRZ|9~E3bf4V3*+Nv34l4) zUJJw#9t&YMhi@xb5Ru5j0mu&g_4%Nb`nl>~T{8&J>VbinhBQSUa<8gvNAPGo^ECqd*+z$0_b;&>MMMOvTs3G^OIA89 z2Ewk{);~C&lNWqV$CsBBrjq~eLQA16ddM20CLRyqxZlo2caR?x_~-svaPdu@n{sT+Q~Bm(V2{L$4Y z5-XiP9@QB1uH!jDE%7&nL5&&LF5m~y$OYUkaug4R8BQUDFm5amyuaWZ#Zao?lL09q zx7@)IxEArk_iw^#{FiYThF+}xA!FpsiE|H_q4#|4% zb0FfOaV$gR!c8k^e_nU8B6&LsRz-Txh=zb@k?U$o;TwU1*O#Cs6WA^_VhXqVGyVvp z#@&DosQt%^pf>cR;Jrn#P;lxA6lw&gA)bQOFjXm)XxE7xp{|OD1@A*j2>f@CjH?g547BlD#3hF*qI=#?9KHo~G7arWvU-Ury)P1Z9i=p6-z8v%zH}W*X zQt4UkyaYNmOe>uZs&j%od{zG?_O3n4CUEOJB(PlIWXbw?SfN0WT2bShebp)`I`>ax z#XD~1G8#)#2**$ZLt*pvH!a z$!DxsR)rCbxa(NypCfDSt{HOKu_1y`H}o2epp?M6cF@@_LV;R# zT?W=h_0T(FO+^ETRlYaq?(Fobhd!O`ApRQ+eSwH?Oeww>B=pVF0q%h=g(Q?*B%qc4 z8E9w>MuAinH75%9NJ(4kOIl$@S3RMvh5-)GeGby$t@@12`Hw8B+EY zn1&Z7p;?(dV|=poj8{adyBURHJOfCHwbEYzt&Q=*_dZ}G3l9YWj`=2ys-m=)5=X+Wf)<@f*1a=BL+ma)9ks4s%tqa49rDg%TP ziJ(6DSV>UuWR^cC@>c))0NA0h$L$a`t{bBMHc_pVFf~>RG^3*Rg_Rx)8X2 z1G+>2E%rMEuZW#dQbJwW#f4^qcCTFQNRq zi(nn-RF_smuh&C=z8*VNm9!kN+9?5@7CFvk(*MJNP{@#?ahMn;wSd%|ibaklr4(Kk zxi*X%9xzC9t?#G*4dV;*e~j3I#|!>}=urnsYqlfhmka{M6X3^hKakTz=vJ9r&Aq zc4(1eH`@#22*L}~>)NIEhoHvh*>RU{lDGL1SG%0{EM1?QTzJLXJdWI86#09e6#0k; zM1tysqz5@hyHKYJuH|M3PFqD3rb*7JJy6j5w^n*2lIQ^cBi47C!!{{c1$#h{s^FMJ zL=Bn30EtMqDXMFdv+C=$XQs$Yk(tLv+CZM^=pvd?gSiDQ%d85upm%UX_x0kwZw#a9rAW7+QBaZI`4G1mnlw`&_fKK_6KvOEV%J)^L*kQGexI<} zFA$=;gTDwCNOuTH;H@QL9`ve*bBu6lETynqa2kCmY!F;8f~@@Q z!lzb8p-hTH&q`8J9F)FKMJ~pa!tCo|Gu4SkoDkAvQKJ=gc2f^?`JWV+e1!2)zc5S{ zyh5PgZw;0%@wkOgj8w;SDI^SMxL#m^z~~A#QYEnB1mnXwb;cM;TyL2fYgcwFaz(48 ze39cCz=nn3Qd%)akQ6q$jTyFy0QAjJpLYtK$ll2-TK;hEZ_Zgr7r0$uiN@<`SAoJG zbqnjKHe={`lG|a4q)7X!dQv>aoG(+F{Ul4YgG<4xa99k|s7=9tCYMfEMvVc2_Y!>e zO!duLdk1^CxcLh2hWzrSwDYWXoZ!>VRgc%~9ei(2VSDDQO3nhYhVE+y>#W)#?NQg< zE|h#5NPe5nR%l&9p=}W3Z5cHd15%r|((^&1o+fmsId=eVyn!EeniFF9P9h=<6T|%& zHIGQ$gue$o!V!|sp^zzZ)T}5R#*Y$^CUTvKh)`D!F7rnaNpPze;N3LdAE77?30*rP zTF(q~L#N@N+O1Wd<|QNXE0u)M8kmA9M=o0%CZm)PtAQ+`H)_E-?f{&^B>aeCrPC^n zO43S?cSEQ6O*c%2ewGyKLCk94Nz^^bofo;Gq5+t_; zx;p#1z3VN3(o|>2SEEhHQwnck%@vl06D2O)-tLe%X!1?rAv9=8O{U0c`fQQY!w}UN z`KupG{;fk}ZzVQTF0b2yW_);mYMA^j$FJ+(LqU5eC7KMxwU?b%|N zS^C;5(ycpCS$tp)v#t{5Q%&nWD$Hku6 z;i!E)j32ej!s{8pY571#taLI;i8v$jx=7zd$xvx_XvGogq3jpXsMk&v`e6J`VIqD` z>Md5rRM6bfUj&`%xqK2wut6re9|)`xx-psgUkbfk;9ml-3T!+>R()+}utJ2uz5>Sx zoGNg(z)XQl1->uvPl2`v*j^ifT?O_Q=r0M3eLx=SOccib0_OditpXil z=tIH36?jtMS%JPXFK#BVjlgiH{826u#EQW`W!UhX#Sz~m&>^sgz@Y-i3QQ3FB*CW& zd|04UihiEP7sww=1+EkLp1=};2L&D%cv2Glr{FaL8_4kDFYs1@odot0SjGDIkh~D? z6nL+|bb-$Zd{N-b0{<=WErA~hEEX7)!6kP)8Z3p^rS+5drm)oXBj|toWS2*eOVi-l zXmV$z(-}1K+_XJOK^xM$lh*Cb`gW224PElo<_KR&Th{gE_<|I*cHw^@4)PmTI$Zf!Go@BWK1zZdkd?j;bn<*w z`byBL`YyCmH*dB_+~M~FPs!Q@*R87ckfFiIx)b={cmkuAZnk&z^82ZwYx3?@*`?7n zB=QH$3h{9kFUL;})yDhqF8}up`8|g0&W#Wy7FLp|Ot~=Anh3oeX{CRK%+cIuwq%By z2nAXZ(KuwK)3|aS-D$Qd(9D-0u4B09)4-$Yr`6Cb&}dY)($hhsamh-b4O*KU=xAZY zh`)hhnD3OfbVHvD8cp}D5j+W+yNC-xr|E%J{$0@8Pz$#*7J)Y04gGo0Lh7N@JlJaI z70`OQp+5~8wMUXSdzh{>nrgOpqp(2eYov~K&!IU8&E~CiN{l;tmmBG3{N@C?i5o+c zK?`$3rxKz$i8bQZXpA&LveLVQPCeF2kMcnL9K8cs%p)OM`&ouE?{y6BvLI3RdMazZW!{-)3lbc)nUr?I;}W3SXL)e(jJ@S~>yR{8^=QQcbUv^`20nDaQ7T!z3bfr|weOUb^%CFnFV z?_`Df0*j?+R|@=4AZ(jqv*7CmZnyB)3jVghQi*V{z>+%hOY=Nz<5vr5tH8mM)=%q* zVvrX6U4gpY;xEy1R%kgHG@LEN7pHx2co-_K3)%< zjt;t$r{nqV=#6irv!j^}2F>oQDSZrdN}QEW2Vry$z&2$FP6ORtj_DyqTkx%Rnv$b+ zm(nIsAbLn^ZWDSa`X^|&xS_uZnmhRo_0TtJy3-QD7R|8Gx7I_aS0|_oS`$h|O-Z-X zV=Z*2&e`UABG^<9o!*#om*d0r(0_JMcbZ>^c9(1D zw*&S6F;dj+piJRq{F>lrrEdd`a%`n!h(KVZK?KI zr;&^)4VCsGVNfI#=7LB*%N?C2lU6#ND5BP3r7y3CzQUxVJ=aRay854m5o%(;zQ`#@ zyElVX4yxu(Si8k6r<+5rVq8x;6P{#TS9W)iH!FpV^bi7#&J+d+-hojgQw;bBK2`Y0 zfEnr-1jTCa0g2&b?;AqZ4&z45csQKe>%2a7ySKPYTORwy@F%h?W3 z6>vyM@V=%Lj_e7pJMiDj_12sSv|qsnsNN~`5xgy(fo}^uAh4};s!)L)vVFI`b(6h;hcogYTx~G|;{_%P zOc9tSFhgLLz#M^j0@n*H6j&^9ufPKWD;PSBDk0Pg^m&vM6e!RkFkE1?z&L>k0+R*K z5I9$0roe20xdPXj5bdi#2t@)*1(pdc7g#BZ|-rDL5jL* z7q9~+P_eu0(VjW`)x2HyF)e)L3mK+C{O9^=hx%ogy`x9|Hs2EavX~D3=|SoKnO;7b z4e9@M|8$>BKm3P(^uIrN-aeV$?|$gl%X?e3x8J9Y24)}eHoT3@GxUGK4*GxZng;kk zrvx6<^ZwZn+?)2my!(yOb7#+b+_=Nr$ivrcyp4)I^ncj_b=l7^v}xs8leUud@|HAE z5sm$FCPamWIkG(tdBg?TSiuUL{>offTwEG#BZ}l@wOdJ{3bQ> z%<3-B~=$dr2sjeV0nOgxm`h*6n}MH^h@Lcu*IF+bCf^?NM9TnhY3 z@mL2j4J15S2A54yY=4>HN3oqt{9>mN&fsi2MG)7N4c>i~@jMxQqVcUx;zv7}zaPGs z@I!4F@3Msr=8KQ+^&2P1oGL=(1$^9-HVFPjZGOkM`M{r#t zdj;1e5P2WV>k`PD%D658Ka?DWshIUqgmHpP2{?^(A?O^O6I|yYaXK5&IY@kn@v_le z0?BI_*Ex!#B?N-bP-X$+IzxK}=M16#ecoaLouf3tb&d|a&3v7s_;(oBIm#4V*Xbs~ zb%r8o2NFSNDCa%K%?!~i5IH-T@ZK)YQEnCEI!D|8W?bj!+*QUKcjh{cmD3HO9$dt8 z1>cH`XQ;^Y1b<)*<0XPGo9t;gNuWXqoJ5Zi$HFHU5Bvfypb zOM+;vfzVyCygw~55Z?Zp`C0UmA41VS=9dco;9Ar_B_M^C_y}JNVSFJi?GeVJEf7D3 z)|m*ecVUCYf^REheg+94Ou-RU`ZKpF*M7$e+Xa8R6Z5@qX8GyZ3?%(~b1)L{zMBOqzhs44!8@E}e)v7i|7u#F&5aYUAmI*$h zy;MmW@lpSMH%kO20VhS{I0thdW&_!RN7M0QghxMTzQ#vkOM!A+mCpR0NEqQowA+I) z+?%`U`nin1vXB#O-oAqPjZY`B0ey*2o99?S--+`r-VP*#`l=dTLJMiz2%*zLjxci` zZ8P}Hq^%=(r(3cAx{)qXwtH+GrNa)6OqHhb@-imR3iO=aQHXzp(8EvpxN_DR9 z;5m@Oe0>X1O)Tr{%Yj1w;u6j$cZ=|fR7q5(5aur7936U!1*%?VTwlVoY$)UUGM_v8 zut9xU&;I3%>q~$3VHrkMrZ4rW`jc^e2~ffbwwLS7;s}r6z5r6tw*qx4X9e?eAEfwC z6gBbnZ9%(mIG*r0%o`}IRgCLPfqG#nMtptSkSET25Uwu~YQ3CG(5Y{NIx(4xP~RAJ zVgL*1o1>z>X1=}`>DV^L^*u~0LKxTgG!?*kIL5j8BDQA^tjgzKam}8kay)KI2ViGe3L{<286! zi{!)BGX5<5OKWO>QBb7d2M0qbj$Oz2dMUyj!H>-02nshc|L!8jleaMb`c=kD-(!4G zx+H)O*CT|(HB$nNV#a4UfK&Thzl#YTU$TO}IwbHS<5izC|L6M|H!tvjokD4}VP7!+ z<2yJ3d4i7|!}9t{k|+mPk-k@iMpGxH=+JI95Gr-5@B6sehEuFB^Xj#oBhYwP+%G~U zW?o2w1kvC{_D=3)d*8gxc(vfC+i(KRD@q{0rx(hf6!fiP9p-Y1llHTLVabd~f6aKZ z%=wB1cSLcOWua|QIQ9$6=O1G{sRL&~pA0o z!V$k~(pdF33oPr(DK3CRp)m9iM-c4cY2tcPSt|I)PcUEKO|z*X7x6i{tL80fwClZC zzx`k?@%0!5DSqS05v-uEk1OxVDNb*|3Tp#dA=-!WBbXUcjtc1uBM6QAaf0udb`=FG>XL}vWPh=(`<=8Y{-cwJm;;Vo=n$!H$^#cvK2c1hD->&bCZgg<+8d@-aK5unN6^@Je_v z3bPZ~fOi<Msj;FF4K6YQ7r^Lk1xVj z2?aeV&FsbaZ<2%Ug1aWE^Z*Uvmi?TdGQssce_q%^h;5>;%`&)-)GKK9wauHPv zesnwILG*<^gyUh1dk;a7D8^^DWP>M1GhY26SSSl1&pz$v=8IPX8^5?}b6bK#&J7g~y-HpPJc#KTA`LZ{F zu7dbU%7H( zKLc|=3S*;~zyDswCpAU=Qv|j5G2ucJ#*L|ruM*r_@Cd>xp1o$I+D>Pv%~7BQ~x0J>RRvA)IQVVT8hd;!LDN`S`4VbHoB|La~( zpyoe2zAouZrfmO0UrOnS?#a_VmRm2WPRo#;2!of(fVijoI_A z5eUC9vo5|RqOO68%DVid@VfZ7tLw^tiAS@PA)SDoue)$Fe&efFbq$=y>Xa1p4KMk! zw$u1wS-)%i_228t7rs>&&v~;hzUOyNK=KS3|Bqqrc723{{p;eZlI!9hNu$yUnEjGA z$bMsR8^-k|J%NRF@t|0?H`j?epl}=dJtaUZ{D3ZSJ$?{#z3cIRZLQ0XY*d%;7gCqM za&KMxD`qh6x<=^C;JOCp|H=I{PZIdXD(Sq3*MVIwQvCOVHR-6Xr}S5GD$2Jesk>GQwikFWO-ld#|47>U^-2Fzgf)R z`9ocN9G30Z^8-f`j`o)?3d!?0g7|bc@N6?KkzBzi{lo_H9$|iO)H&t&&}>N%HW~=8 z7JMtZHQ{A*m_NEB+gmr6@dXb+p4R^h|G@;uU2M=VgYm_$a)hDuRb_;LZk)h!!H1SG zzhoZs$9OTG_5|Zguv0=gE||~wk)fO*^TspSn}SU$bNttLQ+dR53N;@7C`YLA5-DVZ2;$-y${;^%V10ZRQ9fpJrT- z=jOdkDAKMnI_hi8rp{)2`fjUbPjDO2mu!VPn{tus>%LY`WC4Abm7Wjid(HH4s_#A1 z!>PVj?Dq2ETyjsP-!+i&R4Tk}W+P>xnamdE2@HYvG?KtE-`(HmfjhayRw|dh9nlymt_6;7~}f3Ew5U}3&h}YoOYoI z^d(xf$|YRi;x+stuGr17u8kE~$c=tU8BjWQ%`9I;C1m7a|3I!i_2ixB(_z!a!uK-T*8=L-OLih`u!n9nDpd^R! zMdE1o3jWz;mM;@Lb_L^y1b0`+ZOfBb{xYu7Po_-qMhKwp;i zaVqQU+oM8;$oL;w$PskL!2>doCipFi4Q2@Llv!($;EQA>R4#Z(53Z8jciCR^Q5<2J z;Li`@1P8sx{E`W>{~x}c3AY~O1eEV!yzm~*!8ySPwqgVGidaHA7;LJXA;^#j%>kb{8*`ESmd28uvbGRkhF&lc(>s?*LN~@MhQ`b`ZlNzGZ@#m zOuZ{hwWz(Ue`+Ep!0{F1K4|5nuWy^`BE!AlPGi=Dx&q%dkbb?N4K%!oaeckj9P|n@ zsIRlSki>j_eO250Bmtto=M>}m8mm7OSe~!5LjT_p#sqzz)|Zd7g1$Ga$4rh;<6ll= zzP@+M?+W9Y65%&}7|#(rJD>H}3;x6?<{MulgNWZ4j-?jmD3}Nc&1Bb0-;eca8&06U zC+oXRDS>ZU{*#MrP~W$8W|>6r9rJJN!1DSEvJa$638(mtf4s>G`eLruyBXI<2g^1x zt}omA_7TR@e&7h6b}*hR_|at?Va@^OUku_5=x>DYTFdh32W9-<)}9-ozPYLB|03+| z!)ji>|No7Uq(TTG2_b|igb+dqA%sv)NJ0o{hY&)DLI@#*5ON=-U5JJe^5t83b z&*!|a@A>@m+t=kf=P_%|nl&?PZeH`+?a@Vs{6{b_SG;_ZEu?Pa8nprj6lbR)s?}v% zL&O^p8Pvhq$e$)&#Y#>kPp=F1dBXL-)dCnOc!CUp6Tl|Vz#X5#fh_V?1)j|p6uGa#pM?Xm zg&gw02}96=-ud97>Sz&v@}W^+t-2)3iUnAGLj8bpL$@MKah6~ zM1i4y!6(N;ZdAIAjB$TuYfXfbn+{74sBOVSJ%O6q`pB>?JknD7e>wju>D6DaWfnXDGMK)Oy`E(l$ zt9){NSqv-Jde9#j3(lzz{!f*z4K$fsbI6Uw z>Lm+|Caamuz6IoC#G{#%CzB5yMt-N3kjsfmcDcVbSZmf48A4kjVdns_No#OHWpqg( zc?KU$QpuJ7LT=dx`g8oi{$0T3`Cwz*6};{_9Egy&63F#WTMY#o_k%+EHnc#A4IQP}kO6632M5i$W%d{Gxo# zN^2Oj{|5%U4o4p+k>88QXPH66`*0$}@+{etyv-W=>b{&8+|tENguN!>!A7=F;3RPU zYOrT1ex^y5uc_COHQ>!Z>ftbhY$hHgq=9VmxE!#$5ol3Y)Hu}_1>|#HNSFq0_zn8% z=8D_rQJ%QYrlfpLtSbztdu@szLP6bVQR@`AWTj{Xk zz?c8!!JMcHX3C#(6rW5n3}uD_8mQ9<>^}?ggi}m>^MEYRO*;<>KmB1KI1v1o%duq;xc6QdC?t>Kp3glP@{T7U43@-G8FPF-yv6gp7THi<-2-AUvbG|r4>zBvSCEx!~VuD&+Jl8;B z(QQmLI%~mdu}LMX#l~qJ=+)+8~0Z$jt1+v2Gva1f-IAl<_Tn(HL1$7Hn zN*y?;Zo|6T9rFBrFnFvf^4r9Ny-tH;4}jH7=zS0@Hz3l1WU|&g1bu9G2ny=SC7i6* z|6FpS7YsNYhJMd~QJ~)uaI2$WpQGRkdEmrE@b}SR*JCBD)yy7^8pWT0!r@I|&r{$% zJl(!a9>+r_=hKkO2Mrn3!Q^Tha29#oZm_zA>yT44X)4j=}?!0EY#@&!tBIzFEGHW2Q1CP8kLiN)te$iIR}+uvODOpAobL=- zPCO+Ba%q6OT%A|ok+mTYBiFEoL-wyBmpdR?fdKNWFz8#n;rib)1s?glL5At$;j#WZ zaKvk5@csxc;KUS9p0E;f{jZQm?E#y72Un{O&LjURdHF0(TD3o+uvMI5Ne^Rxfy?nL z6`n=lqJ3~Egq-#k2BXOHxEeFDTyHGFB6K#u480sZojdvlAY?)aKGuEf4p6F~(TrP9NE zvYM%~bzwlA;kZ@+%OH~aMPwB-nH3>F9}RhgKG>o%*uD~2PIM*fzpeoi)J*42u5<_v zB^g5g_c=Jq2y9sk98Z3=3v5{h@;w#7_T)P})(fr*xwD+;{Fit*mGS^`pRWI9r`iAI9Pv0_^8Bwdggc^ubQ-Ah6a|>n zKmqc>L>AyeR$Y`%R@-U^6X>g5wPLlaPNiJzs{bgbA5JCb zkyV!!)`na@oJzTC9k2nnw%q{L9Ug{p)}}jg#>wsBs1tX2cK%sxt%;< z2lNwKLT=4n@V8cAudZ-N!|SNOvW5P4J#_=fc|EmO<4_Y##CxhWaaT})x~{s83s_xS zy@1E{8aovDjobEo@>}t^Eh}W*9`Xr1baEwcT*nHOuvq`{%b`iqJ0OEkIuvX=f{*f9 z&&eKqy9eZMoxpQVQKQ^0;C-7R*K`G&@&H3!Oy@BhEnIRzohVr zd%>YpEw3NS@IeONccj^G1XlMYofF4!B~4EK^+90u3*F9Q)JVCyU!fcK6Y9skGsM^S zq+DIpkWd7Nw5j}w?`}icMC94VV0ACb>t)EGe&~J@PqEbfCM)!zpTi%*4}S%R{&z3R zV93?IOcUmm$h8`$F=*1lg~*UZ&gxBr!fxKcd^1G71im!B(JpWrwfx^}sP>3P#NQS|D@|rZrlYOE8!WjiPPX$kE56&cS zZw!5PPs9Up1|(Zp-SBs&ifDhC5a$O2ZEC@QhSzDt$0LKfUSpql#Uk}XC~x%}a&@O* zXm`j9D9`Z*-<=NoXKp~>a|Sr(m*{_KFr5iq>LNpqKNMV6A%n+EaGfL2_a~1Ln@pMC zVHV`!rI9~_{I)dm8_$M(cN@qf$Z^_9PzVfw!r2}$c!oUp7v!1bni1g8IneJWo(rS{ z9&^E6Yao9rxzSOu6ED~pdj$IGA{Xt@Zcxakg1#FRYyx3$Z4ua!93>t;Wr4Be2(cfK zoJk(=59}q*M}GUgV0B}Mxj3MZ`j)uCL#ufzzOf(^3X!0Rw?z}Bb7~&Rpo|EH!PuJ6 z_gDZfZ3lx93&C|+Lytw^3J0Jc7y@p)6#CAKC5!(5D4y-41362e&@~?l9--i`@4+T3 zz#Ydz-()rAvmigS4m{}r*fRqBj1QxQVPiN z8dXKY@5^X1N3u~vWH2^{eB20dJlXLy^nI#9?!<%2TylA5IAmM{^3(1p(1ZL1Psdt~ zN@zV91{L?^`M+Y@m;aUTTLZQ*K>r0^qiZ5}TyVZfbX?x^XyuUgM)UONqlJ}5jlD9VH`~UEIP)O;_pfLk~YXL4a z2bbrj)VmS*??}kATYw*aMW3g(1Y7t+p3@3ETRh0fQQ_JZJfMi{f4ID0SA6OXg8@rt zaMBkV=nu~80J-M?aLhT#16{#&tT1Xc)4&yjAUB%WZ&O zl#E+>ZLRn;?STSA=Yt0%fg^&zS@~f51z`2&G@f@Er}6uP>TY8@@vJEA>#lKM+H^P=K+YQj&LLNc2HUTP{*_9oK>kK>U-*QXAk&P7i7rV2R^Dn0q!Tk3&g}DquTT|`0c+4I(zc%x{wExjjls(ai&CH ztBKkP1s^16TI~b-orS^rE@0zi@G+iZ1zrS4i5g1>63KH~q5!)T$X)ojZcmO7T_W{$ zQl(ss{|92vClj1X5;U4iqhY`;4F(pAO{SD5TmtKjg+qnp6du#rUxs{t2iOZEpWt&r zs=OjueCBRJiiY=P#(`==0 ziUJCMBL9M);OO7r90%y9lKsSLCu^)OgtQlrYSNz8{|_>37c-ko2qeE6{9m?sivrRq z|9A{+Q;huYKZB#l{&&H7BnfvPK_##Q<_0@>p^fxuTJf%k|&85;F9yfD9G};1`_9a>#ur z!GRbf$p47=imRUTNN8^fdO^v;tm(E zy0!6IDw~~w#N+0b9jqmL%utXHGzW#)kXWup!2DTgvsOJp-vs} zSaD1y<$2_^L7aH%LOxR**UJ{juMd7Hb~KXJt&3f!qJ`9Ty36=zm&EIJ=k(zEpWg@u zy8ep{Hu7>?@oDoI>@DxG6`wQ?OPzM$_IwNGXAAyPo^wCh|2QfXPWBN8k+MP&PX81QsKhJiew2q8y!&_VJCZcuCnLjUAq$ejm) z&$0^~2ZNh*LV015<@#q^15ukk0}AUpf!$_7YY3xK z?tKuf-UXjI1l|z?`>BFu`5O60VlqP}6_$j9v&s9Hf%C}Di@@K=!{>uF2`Ip2HdvSZ zcN*A=T*(J)lTfateBmh)+^7%~3l1QUi3SIgjU&LJ;pFv8!BONs3&64D7W2UI z30inOb|wd0dO|?z)o;JdBEo9~uTWCAV4vw$L)cI2Z}m z`A)w&qAndFCD9qo=nKG3x!zgCmQgd^f;RgT*N^6bfA>*L69 z{|VUh1bBE1Mn%y{aK(P$h*RKA+}0bN2JhmI$@vWU;85wXh(Eh?Na!sV59vTI*`YZK zbUzPy_d;+k`FSD=j7x@mPk+eWE`XaELvD8w?7|Zj#h&7wP|5lqo&trP72tqRna@|=kXINg>7#NC9xk(in#dt`#=Oo6`Qx+}Q;ds3m-VJJkC z1LDAOe{xvHDg^Y?&Xw0+UFe7mAemIm<6>Pz$S?g-xiT;q=)CP|mgaX1% z!6SL{nOYCLM-(c9@>`SgSpVlQMNl~MGRobP!Rm6#*QbyniSozdSy8r-xZ|><1J{_Pu!U)6C ztvU2PXP`jE#+Si9Eg;uv2>ocXNei%!wH6BV#GXwy$(dH*(;r}yLx}S+O6GTtvQC26A<8QTFH(d*b|`7Z`bS@yH=t??Q$MZy3D)7Y5xX zf=hFK&mwmc4=S=iv&oRJ6fq<@ll-0UgcNTrnVLoXnM^^3-4hXXp5zr}(IR1fkSE%r z#)Z?t?aUxoH!rQ5j21GT1^JJ%sDLH;c_i3|yf;a(SbsF80mu-Z4+k>mfn6A-&Vk_H zage((XbZ#+NsgM(1(3_1*OWE32myZ(75abqICoSwl)LTZ`tQetU*f%=%;2yX27>Ow zfa4Odr#N1h@(6OC7}t`&ErmRQ53d=^z@HjJZoC})rV1S**Y)B0AGZ<;Gq@a|As-Sm zn=HU{735xG=9BEQ8vI}`95xCEe;yA9^T|c2&^K8F`KtNQ4_gOLh+=`Un~<=FM=s7= zz>$2O*WC^F6VGna0iV6#_8n1x)qe1|v2Y;j0JuRfaNt4kB_48F9|jv}d2HswgyxH2 zFzOf-T*P}j*~cMC;8tmndz=J67AK+7p)B$&juOTFA41>l6!Z`O{4ZfQhH`m_IGt@y4d3pj%Urr&{r8`+REnc_v_ ztXCGOSp8nGx)pB!H#n4d7WVq2z@Fzhve@^_{1N9Ri}BxVD-<%5p>R{|XrzMT-@m~6 z7a`Zh%%xGB+=;`Ix73xh0jpc<7QO?kOXPI;q~ww+?1}OJ25U#U%VX+Ix z3-Au{@>z1|8*t;+XpwaCU$F_5a=$#ttye*A@)q2AA-Is-)gCS4{EqAY^PZSW{of(O z?dmLmoY@TQ^d9mZpV;L2;I|sYL>PJJXXxjUqprij+z-${D84l#9oGNOsdPg>dRPbr z6CO6({{+X(g4~~6rVluY{5c8+)5-c5AkQHCaTGcHg1z6Jp&y`SLY4AJNFs+%2YdWR zhBeNpp-~a|k=R7a8YGb`B!lh#K)&q|9B?QGJD!6)nQY5vNp1RHDEP1mqsxc~ck$WE zuUP2of!*)HppHIRbwQi~xL+Cy$R#g14|X+zd?}v`0;_=SVoJJDtI1$OLq6L%R)xYp z15se0{Eb!dslX=otPbAc33+G@GGn5U?8rX1F@fBi2bGFZ0WtpksPM8HnmC@kYa}?a z77R3=fCBT##dW||wIRQ61a=^+778};lad%<70V)L zRE55dHw@~C#YMJY2)U0PIEiew3+z1+`k|4qpI5?S{BITCe3ntInFNJeVNft3|5<|? znUc#cf!u|>K>R(R%1+8Gvaxxt9 zyg_`=R~8sWUMb#+Nxn-SvmA0$ALMVi84mf7?*xKF1k3U@r^ApTnhK9Uf|JQ*HXuVT z*`NFP^eHGHm=7uzzTh3Ku{-%5|K!4({D3X)@2gu9=tFt9(lu6$`E9KI3`ycMr>v@*eX6%sbhfPz1H=M-=Tx$}Oo#cJqR*#QnDmz6J_ggvKl)^H6t zl)Q5}IARUt=R(Bqf5{9wYmx9Q2nz1x@EEY`I>=R!g_G4VizllX$swy4(TzZU6(g?X z-;>!SeoRn75=K@rkw8|%tca{)#9=)OP%#olRxy%IRt{QiphH?ltrHW}un8oqVG}`C z4xS;~_@aff$v@dec9AGR#YBYS70^#0YgM0~VS?(zbnkCUmO`Z$;Tzdqc^CS!|PkyQ)1lhwEnB&!xnC97d*yh+*< z8E(d<)uvKse+WHs*N$#UGwF3S{Lvi_=Zui1hE)wnk!s~B-0s~B-6 zi*a0H(2cByu|Ihb=aCq4>OSaaY~lL9aXS(;Tj9V1AFvHs4Lc{Y8g`y!HFE}&)yx=6 zR$Y`%R->$F>;GN-w!r}vL{?-KMBZc-BOzqvU=mq5m_t_cLJ?Wb3$EH26sQ&tKeCF_ zaIy;0Gi0?EWRXh+qs5H3!=7q^XmT}w$Srn2u3}E>!2}g#p=1?g5oA^4RI;jZ4q1(g z02Jt{qy90~1tGxsp|leaNbh1IVfcLddFy;bc|AXtEk5abyI!i2qC` zsMV~Htj4`rENY-?=t%A_zcV6+Q6O23`|$sxpGa2oKqk2)2ITsm&jeLtqg^OaMXd!{ zMXfWrH%EyZ`2<%>PqHs(%3yNgE;t-bmQ!)b`k%xE)ug#(6%$2dH3}?t!$H+TcI5xH z5LqqHZe&%VSh8pltpDjuPzH<0%An;Q6sQckkd;ACvRYgM$SMd!$SR1#$+hQk{f}mX z8g>a}RbVPv6_`s_4i^3&eT%sNcA+y_%?lpn+FEWReVCwvDuApuks)L?EThOld(o#! zWL4u_vMR8EtSX?{3kS6-s4SVFYUE5-Goc4r&161g74;Eh73GO!<=|biTK$T^+W#%L zy8Gaf%3w-XF=04yKdURGUXuEo`!1mM6x)ntJU@5;PhW18!tB@%WR~ zL=r((%WEoGEw4FbHLnzr)eg!pUN|7vU$te5k_qCYwoDEOz&od-kNwGQxO_&DRf{B( zRTJluRSOv(ggq4#o`SLd^T{NE3hKe5fUIId{}2qQATlMZ7;q&k2mHxuSca3;Fibqe z^HW(z$yU_sJY#htaeOcWVHiIB&#lRJB<7)#-a{uk)VPsnF^}LmPa60 z^ME5+wSW&zk0)7;3O};yqCl0O538YMwIK;P2K#D5l0;?~YBdE+P<@zs z92wN&<9GtBhFuI<^q1ppt>NBth(SES#^QgDLC}MQ9)M2)`P5;bAPgG(FpSY zp8vC%pjx1atb)+=GzwHLVo6ptwjrw;x{{T{{$v#c(PZ)Li1j~}32G)RAgew$Jp%_+ zA6t>tu=F6S7V#mgVI1&(^kc{ymvQ}1W`YWuJhBQJowIO2jcYryY9U9miV;t;ijg?7 zYO!>(t(J#M`AkqvsDBOxs+h1MtC(;jt0oR0s}=}V`8h13$SNk{$XYc{&Ca7h)kmIW zwXF^ztEo4Ztb!_$tTre)WOV@HmJIu97zcy38WnWWR8UQpNmeb8PgZlgZJU)MZL+YMZZaw z>#u6UGpWd+qCA_dYNV3}xmtWo$SPJ{jp?6{W^kz$)lM$ZA88L{=vrMdTR14=}ol{AwbzCaa0bMau-$ z#C~Me#PREt1i~YFhMy`NLCIQXP^Mp$2MfurygWA z3ZltsqDdyJCO5tgdpu(j_x~_KMZG^+HBm5Gje=;h8U<&_szq|iszr2epg`3k7G#V9 z5&zCiP)!s{RxJ=gR#BZmRxOZ0RzaFaRxM(56Aq|(!H!(gLUR3gWrB**K(d-BqR1+U zlE|usa>=TNzL8Z6S!col)k5}UIqFKD|J|9OhFvsS&E>gdH42<>!Jrxi0c3TK7*AFc z&0X^Uj`?mQf73Nw|Bab&m2agi$m@B6VMA6Y9IoVKz7z5%kL5d|2(mf>i6MK@{u#3I zbgutdOfct>$~W@&LwNKux&x07ZULK;E3E=slI6XC@=VBvY#a!7A&;5`_8|vq{g4pK zgb2R(izP4Pd%tAzF23i>AQ8>o z*Knaq&ZE|7@+iKdNgzL93#Uq!>+fBje&;g0LCpMuyc_W+LgZ!Om zLc!#x?4odT3cDmmaLM|=iy4xs@SIVfN&dzc5UyEpXdGWG29lFmfq3#|u72s{POOmS zL*$qD0LoDk@{sF)N1kxRP+>7^lt{kH8f22+aNHM=;}}DlM<^hSuxj$P>_c0ut%6Z3H%Y^6b0ynaG zJenwge4X+Laz8e49Qh7gB$e#X7|9}UW|J3^efY9kYnly@T5;mABH!UlCI@mA*3gZ7 zfbaRd$qgAJ0c3fPpzPx)@(#v80y!Qrqt#?H;V5fVKt9fx&^&>Ik69yQ@=3OcE4i^c zB_mfzfCIT?{oP>wr?7XiWGWZ&Z_R}IoXh>l;eN;vPF~9xNGEUMM3hH9&xuCo847qQ z-|veSup(dN>wO3E{xBhz>%TV>TCqvP$bUIeB#>wERcaR5XgLbd%|U@>I8RuRe=+Lq z$>lgq-N<7_i72=GgW^UO z8x%bM2QWcx!Giw}Cy>=9HI=NkWOvDG%au!354VM6wWHF11qW?>IS%cZpq>Mq$m;3U zn_QmDX8>6p&4!ZIi^~X=pYuRGS)C0hlbdQeEV7uO3dkp`_1pM0JRY=FeAlb2M)UJ6 zc=I%{|2yzEG0{l5=X-FkDDX9MD)l?%L#{m_`cP)j|A2(LqR%BSAbTGH|0FwY15f%0 z{bJ_t^$Gky3@e%cIN353+^PWbBVyD@`F^lg6D(#TnXv6M6f#$XD|`We7wf;2&m>Q0 z4IYx4B|_fgEA(^sfa`t(|H~GeL$;#*uO%$T|ISEc@cfPpOOJxH$)m~r|Ao8>4PGS6 z3rD4cEq_2>nhtFtcct905b{okp?^iPTz_N5dpT)9{}&XD#STXDN^&)^`bkdw4Y}uf zaJ?e%-woj9Q~lm{=s+YAOhK z#m{sx1~!rJikVN!Ym|cglZblBbIH+S>XqEHG~`Rfj!N={(iKYfADYTys+0*$%0MB8 zQM{gPB^DDY|3SVe=61=Wb)dhKLH39|O>DEJyhmBcGZ;fP%7M4C1=f_)B0*2=(PV~F z<-uzh^%KdPXwXR)@`Ym0DD{t%--)?ga_b6^cV-us)&sX@7fdHNVi$eYGT|1R*ta4w z3=ylFH28*G%qAY94|yrE7)f~w*>5MfOC`uRv5Slh!12;y@%OCh4Z$yE1qEx(njj%f z_K`@iB;ORrYLbJSLjF*W0>Sl~fji5f6l_QCCFc#n-sH%2U?Xej2g?=`@@nLfG6v-D zPgG~ZcR38DLGlW*+DZ0p0|QsZ`Yt(~+(WEpl9S1GWlRY6uz`L%v7AfxBkvN+x#R?L zoE(<&_a~;cMTXsS+zN%5cHl?wP;-&IN9_5e{4#mVTyQ#BpDpCl0r>~ALet6Z;-GKc z5%Mz;g5~c|Jnf8xX0lHO`*#61JOsY)0DdWhO2~`Jx9CuZu8@C{K`i8b$!2n<6l~HB z^53-YMoth8Yo&ricPQMDQ7PD+JW)=ig8j%IF5+9Ug0mc<|B*5CkgSn2qmT#pfc(4I zGfMtNZWj+W?Fsqj6=MCD3Z0nHQ0!bD~Iq`{G7V~(2Ti8+l%J{DIY`r z!2BU#t!AX08im3`D$JF0wcwV{FtD2r>>@v-!~Oa}Zo@8mMDC?(+#m8j3{n^JPTFrc zK+47VpU-hWxFkWNna3y|pJYf_9dxNVZf93{o1-Y7vKl2E7sfhKzf{X!?VCO3Slu>*_<=idUg*;cT?}EL_ zbL7k@*k}m!E#=%USa%rs!2z%X`GgEYnSTW2=jEsq95+IX1V1^BMM5$a9?7{{8XO6^ zc{tdFyps;Nkmtz7N$6XTf_??LgA(jUPLdlO!A@f!A0%T!u!oijKUX6mlbk2VsZhuv zua#4=;6k#cTwDYjdB9*T)+pE$>_-Q}$@S$Z5c;}akgt+^NWoeYCOEP{YqF&rCPKl9 zyhkn`g1yO|ud0fNUjqOu_-PiLm!e&KrW=$kXV6W-{bKB`q%E&x8q?a@>jx8eb?xi>F=5cH|g2 zwF>qlA7IdWPlbL4D-cXhVS!QPJ?!E*a;S_ktuUC(gfR@dEON1r7)G){KNRrkDmah4 zuQMF-m5!L`tEaFRMJ^*+Oe+&|m|z|ah42~3;GlwzJc0%iWMR_K<$|v#nf6{>W9297znvDEjE+#^rM7}Q92ElpcGAy8w z>>_tiLhdpb_WYPXlRSgNuz)P?|3n7wdB`w`QL70Ao66-<7_=n6ksB1jcI0Te`pE*x zv*eSLU~lpwxt0hHAlH`r3Bk!^tzi%nvY1eVeO^Gm&nVZ-heNM9?(N8#bl5Bi@-(># z6$Mz6m&-k$U?*}%&V<2Wt>!8{j-rB#+(3v73FI8PBNCiW9?!WvhpZ>$@-zpfbo!so-$(F}7G7`5Wi{Wb!6Xy;N$Mpo=3X%V#%HgF^Cru3oweVXz*@xfyvHW5AB=p<;l1i9zeJ zkn8^iF2CVaILmcU^rvK!X|v*pk~%ocMZ7h)a|zyG@e3OVezRNp$*&KKw>d)J zj_fOc*-x+oc_-yg~|2A69!zUa8&#%n`94iono*zd8wH5q&$Fp zWUQ#X_${1F@{>QJvVyb7W90AR2+koJm_eRb!eU_Q4?}|4ZM497`FHLjgC#jZ{$z+? zYqFyk*o7>ALC8@2U3-}LQ6~9$E?Qo4uq}9)2qnqZ;wOyc$3+w)IYS&%l`L%Hv9u(? zNYq?@8jAl&1=A+rMDaK(IadnAr-%k!t-xtwa7uZ8d+=%&5G^*^@{>XK><5k&4vYUQ zEoG5KsrnYB-%mbao~G|xx^%+@C)Qrp--^{T#AsrQlduj)65jJ>LVw`o&hq?!0< zCH^%R|5}KDEyX`;@voKm*IN7&pWrqXBL`j8Z}I=;t^fa<_e5HT{`eM=C(9a|ZuqQU zHL}-igGz>NCRn$(9@}n$O`BGp)?-_psQ+8PvQ9;t@l&UJwej8X@C6maenU$)Z81`2`tU;vCS_A1#`3-I&i^|uy zYAajsc5Q8K+j`lJZQH)JP3v~nUXcUSD%C$xY+kAH-pJlX`o@uFSA?k^<0{n?#ddvb zP`8T6WbHM6Ts!Lt6RQRbeT~5c>a027*6eD_U8N^JtCzMcY@4y`-~&r;R_oL&^YPKoZx#-Dzj$k=>hF_Sg-3Y}fEw)uUCH&Nw!(#hJ7}CV9lkrRFI$wXJVg?LE71pGT#-Ecx3dv2QaQ8Q_4Yk!K5SRejsvcVPha}44;a2G{YuJu=fTai zhD+YLed;r~qrKDnlglnH{MC^gl4hJEH!|;)!0rn};}0N;y>M+t}rh zneT)S{a@7lGH=tom03^!)N0l`MLX*6al0%p>lTZSO>eZQ;IVbzQ3+4dzMEYgrt5QN z{;ellUpS_0xuLy(w(7&kTk&PeMD4wHyoT$Ibz>b;%(UU}Jxq*E5BaWMxcWs=tFr?J zdrl0_-#4v#)Ur9wp*^2JJMa4T`J?sU^Oygcy5)7wkp`bGRd_w3#(?WxN37Vq@I&3R zmug))X+OdstN8K8tHw2T9o>tZju_X@Ic4>DX1NZ##+~V3+ROKEj}At$Cr7L|FBKBh z%kE>sq=tu}l8$>l5#PbjzNkUBONRE$$z`wnX2w=CrMQMmErE>9ng`V!lTE zSZ{f^=|2O?+OIhDFk^t;==%PR8bzPqe|BD-k;~7uT{rst8k2crYr57s=9YEqNkrkN zcTK8=$GixCyr^mSVV5jw&g`A--KV@qs%OOpr=pKq``;+j{mmj>$B)BSObawKt$weG z^UdsLBWl*{KWkx^Be!;r9HP6U#)eZ_2c~-Mx-`3n`$}!EE+;Ga&N&d#KDu(}zJXP< zzWY?!+@a!yY26-u9=Ni|xYnkm>T!2K@aoQ6R*z>&1@MF-Fw;f;G?g`jNEy+?DIc08rK|@)6;57-&t;{ z6${sE2RFJs_KxP@rq_pDZfRX!Tu7{}=^3u)KBnJh?;GoD>`J?}Wz(vw^HyEz+RNqN z1N}=a?BUhM*tq<#GJ~Ggx)gTspWgeMwmqs>e(Lqu>p{O?=v&{k?HW=&u3_|_8-Y0~ zKL*$(`MfgXfFtjx9IlmJJTSi1y5tszBYd`fw>faC`u-Ms zwV_|!KaG9)x6gKBiV zKd^eqq3fZR#nwk%6Z#C>ZQm}&D5Ju-bI<3E7!3qxPJuaWR_;^Hf-9>M$#~i9~wQGeDrfqtStWx3fv+n=4Z5DVTZt!%ww@bH=sBaV& znK5ce-NHu|YkVEqy>#0T_B*xTl3s;`I3BoS_TiC#_jc!t>IM|GooPI-YD|2@qS7hd zOiUJ@`x%n=Gsoaaa;fot$)@_RfA=%A-T$V4nHuFYY9{4%P0G%+_;BiL?(v6HCoDO( zW^}vkS}~h#&1=^3F8{O5>rEF&AG)0$GTdW+vmRro{Cln8-|UYc>bCnZd-Egzk9kAO zYNN{ct2KQ=oAdie>QwqQ@Jf1@G*?gWE@iE9za?G0+0HVy+T9DwE*`8^%kS~GxC5KL zf@ah?yJ>vKkv5~RkJx;D`Zgo4zPe5CjchZ#VC2`>RyQa7^J-%H^{Nv$bWeNua$}pJ z#i8SRS6lwuczI7l+hvnKw6_geI<2eQkws79jE0o$df?(;ZHVURoD(%{w||P)e>MBj z&G_*4vzsmLGjenr|M@L{G`_xM#kB2BwuNnO`e^8(uv)bZY=9(}{6eXZQ1?@QOl&4cPU(n+~91*kHb4^?PL_^7XEVcuY^p z>TtvTn|<@Kv0qZ&*0)%*E#!x_)wm0m{r5ezUvG$x!UDMW; zO-p_7&1&P9akYPMKULf3Ouc#MtZO)*>zCQQs(Zwgvdf*SZ%H|4GVfvg{l9e{pKHIz zrJ1%{ck8QHzD!wr>v42kYSba;%ZHW>wfa^oYgzDvF-MP@-U|%!n9-|fxZae(1D~yT zTH(6*#kGBg-Rph$aN@I`d4rpera$i8HLscFy!Bo8dRj&I==s8S%G4NbnO7H*(`G~@ zdX)>(uJkxlr;GpX-1t)+3hH%xvqUfQmyUm4Pp580_ZRHe)*n!~$7{5|cW_?j*=+*{ zH=mVS#eTE#@RYsUSM4XRwei*Ta&mvX|5Wgdm|uEV$5zQ)J-n%7M9#plif13TwL4O@ zCg78c`@yuXtIAJ!`=MV_jdiQu-WxG?W8cq1vX@=+a%i;N?qs^_vzw*fJ+!D0QORia zig|`rD)jr;wA$kQM&C2ruh#xvad~k1=n9kS$DUug^m)&*>v~UIb!bTRjvkHN7PcKe zC~K4Z&P$6bv>UK?ROMM!OLyyc_T0rU2^Wety&m-4?OMd%b-f?_8nyn}yg1`2^_u@K zHmva{rLe5$73|Ja`#kV8+cG<@<2~0O_j@{5 z8oe{^rCzDJ%i=$;3p(8OpR=!vjTdiik$tN0&G-QZBMw*T-*I1ywO8BLDR>gv^W3>| zT`ELOzSzq3->7Zjbw@lOeCPE1F}AD`+4NQ=0)qZ6}uVS7&oU`K_aN`gIB(Hrnp>tY>MDj?D30>|1lG<*{{(Cplc3*eU2mL{R=Qe~U?tTAh29 zX6g0kN8NUQecomjZ`g1$&3jW&zE215$?GDvG+8_#@W7p>L(iX%DNMV1;d#*1f|OX- zk*V5Wmt2=^t^PFfY_-bU`{rG&(z5eVht)yf`i+>gXG3W>zy3ET^zS%y>Y=??=S>=U z!L4|~@)5VDUiffpuWysvk9v5SS8KNG-oD+>y#LYLn0R*Z^MICC3Fn{MXk3?f?Y-eq z`p{=Tx-{=R<=~KxmpV_q9$t0&NQZj&nwM(R!F#Zq(*mOd?G9)oLU)B9$Z(nZU7Om#lJdc}#( z=V!iqJvVdM{#7YE4NJ`kF{q~R>+I=LaQOV`o3ky-uX6DGwXu8)$KL_(YEJ!_ShoH7 zTJz7w?X#-yzS+`tNbUYxZ>QBP>bKp}>sVDo|2nf5)tJsbu!iGE?8dA-*9SI^9!dARSHb9DF)uZG_yoEV^Yea9=8gRgQs+HO97 z=|j)`HGbCgxV_qQP4?3jb8h^+SbTcdb(_|1-F&TTjM=O6sG8QVc>ASxjb@pf$ zyD#{3WZ3C8D+bbyIXF>YJEd2P&=r(`nUo^7RbrIlhl@Zk6@+NN&?Ei%nvaXS_VvrrqE0d0ET8yd7fL zdG4eMW$o`(b_?8p|3lcZW^G!v3thDR^TxXc`#*c-X*I2f_#db^bma_xhwMIc<6jsXY-#gmc+~F1wKqql&hnU+)SzRZ39s)yTes#^ z--z(^a$91G{(a@^nLTa4`L|YelFvFA%zU-v&h>hcZBKspViNq*P`b=(%PT3{QWbwN~^~fw;l#( z=Zu(M|3Z-Gv8(OB7i?I}E8lNh#JQIl#Swc4-F^6VUf$B41|HS3{JVQTtT}3+b;WgGYRquE^u>1S zfF5_P>wk3L`1#$P;BQ+E&AG zklD%}FMQ6&-l=!0@VT48JM*4n8tH8w9}p7KamOC*#GsyDR!#QjtXr9W^Xsb_E-Mc8 z`%!P&lfhpD=QnHPoS}8DaedP8#sh9$D2RA*b@ZE&n|yOD_Aafc%^2ubKdkX!%XQzk z8)z#XOCOd}xk-g5CWoso+SMa{?b|jcb+@h_S$KWHM*9bO&1ZR!f8S=P>+RaV%q-G% zoBi}YtJinfnv@L&FUs3mh3+&D9qLj$rH!NY0ke@yo^1cI;FEomEuA+ecJDBhsrRdsozCFItwL?*#5AI7Y zr6rUvAJM$g8#l2?m|sZeBj)K}7py55hFhVxrqDUf*=qz|g9P z-z@dM{kz{*zo;z+RcBT;@=VtDNLc&VDCutM&9wtZgjaiYBi{YPUB`FT$2A<#6Cb zH>KaN`EZT*c+&>@g{RAZ*P4AuO`fwn2JV{uNcbpIiH%#_De~O8>2>?e}BGLhFmMXO5R`Ztt}9 z_3d7B!shNDGB2g)=DiQwKgs=Wlap_fT@e*2;HE)YPa9wApJm!=%;Xt<)#4unPp|)H`rD7+cj$YboP2D#&xjtg&SWf! zy6S7uYIf_Cv}1iDwv{TE+qs3dLz4*~w%Uzbx+V2vvt1777e`l}z9D&1QnP^-P3lx! zkZHfE*3w3E_vIhUocyuLo689U*X-X@7_r-K+|F9Rj`tmuczJc-6w{{{kHsHb+wizU za%4!ozr!!CJ+W?r-t9#btf%zI4fok&SgF-b#}5Uz1*_vM%~x(Y@i_9qgq1(czQ1;? zWT>6>-mWOXR%y*nCQ{ODe-c*^p_4<>K_Q@QK#o*&cCt$I6r?3%^qzyGcn zKk-P!yI-be_QrbK&fA<`e}A0ifWM|mn>Q`^o_}!e2%FXyDqY$<^kG<+k3PjNeq;ab zt6Ryof6T$DZpI_;_3^mAuT=k26Lt(8exOs~*D+0#Zhml1(e~Va?9Q0d4kLyw|E0I+ zaq|%)79_=1o^xQk|A7x_&GnpZhBzcGZI!ir=#A3bujo}UYw2!0G}&{~-<&PIYt?zw zDdm%O=Y)hNTdVo+4EQw1tDt;?+f$yMADw=oX8wb?QX6a<{f>RHFVgPviVAhNj(k>T z>5B1YQx{f?sJ3nRoL4%XW{te3EobOwSbM1U-ZY0+Z`a*4EbyK^pz)#m)4iOEdk0;x zJZX3(v)Aea!}d&^+pKJ-7e;gJt9-Ay`)vklYNw7H)N39Y zeWab%Y}N6KF{ON;uQIwaD+eV@!3=TG*0`C#tA z5$D=`oc(0|-u~BYH)L9Ft-CgGg1s}@lnoP8?(r9(+yg#?g$NojbH{}eD_nPr~;n#B$OO5+wRJV7B z*7|#z-(Bgu{q3!ws4DB?jx~RGB+$LG-C4hn?;1Xu9D6(A>&=rr>kh5Eq+12|J_&7B z|H|vs;KtgdU(F`IwM@T$byxFenah5dEGz$H?!!G3Zu`zHJ2b$5tabCqtv83Hbboh# zbljYkwYBX&XO^i`s#Lh{v4L54ZJYFN`1XitZL8|KrrVt&EE-gNknnlS^b;*+8g+HP zS#|d)tN8a*`{~~PIV3m#nl`ieS(kR6GkR`tsx&BNlJAKzmN!P$The(+jRW8GuDq^a zy58;b$G$c_(09?rp7W#U-FqDW>d~zU?%vzdY{s8#vO2q^;jXI#v{@AzU7hNd=93WI zr+3F*13!Pvzo*ZFn(eWyd;gl1*8hc(UYkrNoauaZN-^if%3synRRG_P>t&-!(@ zw062T>iE5FD{5|Od;MsYePjOl{{rp^5%<5;D+Pb~R2)SWyHkKpB9`nMy2z$Hl%|Q+ zo*RxpYzjU`t(1^tz)|`Xal1}X(vs9opQ?d_KH@q^0Zh}1@ByqkWZz>no0QHOOCb>U z*8`59=)BN~IpnmPX(iyy-I0A3(q;UMAfDla#Fp@O*7kIlYSE3u6&YQZn9H@f(-f`B z_X!PDp`N>GA#j$|+z4{Lmw=QHAb)ZpT~A>|)n?BG$n9D^s3{hrx=71?0gyVvx`QOX z^)70*ICgi>yID*e3%=I^iBTq#9-;f-`ah|2crzWVv<+>ieG2D*wsyZd6)wN1>*J;{ zs*HVzRTD3KkOYT7qIB;ZXGnn0@I~$!;h6xELc$y@lqmWt&D(rd;n$!9~{Q?5E& z)ls}*&t&9%3|S8e>1c*PE%su_{m@ms7as+|E0`A&d<9swO^-4)$A}Ml75A<{4_c^> zqO_PF!i1;%dXOme4AR+jY$B9o+>hXiK3`pAlwGCPkn7~CGE>n>2%|?4j50mAusCjv~^*6~awL<^m_`s7! zIBO!*O0m-u1%56^Rn2tZf^7h07Exk33ObQ)^M9jplOgpvi*UE5o%5Dm8ZUCXmA1dLjk4$zHb+YhB zWq<64TybHw1q}h#G-dN%6ggl(c9yR2{VUSf+y=0>vdajgT#3GdmL$!q`01yDVVoGr zUgXBh!oogUk<+?g5vTm+}p7bn1 z2bw_E4q=T+2y1wWZK%l?N{a~1WY&9(*?%PgXAvf@yb;kJy!V$Jko!G?FGZ9=ocJ+` zbr#jb2+(^lcns)BuFa4eN;3J$JN}n^2f_*Dk&AFwUjgpE<$~=a!0KL;uLcn~k8XDk zOlD?$>YIZuka4u%nhRfyU8IX9a7b@SY@+PN=YcmPSniAgxTYvmTN|AQ_hbF+QA`+i*;6^ z%Y}n=PNO%oMG(Z{J0I>rngFnXf*=Um$sW+$xYAYi(gi6Eam|DwA}pTlyVSJ zH#bXvLw$Lq)-%wWP5eCe+Dr#L@Xeg&Ma!ohV1MT{({h*r zPasqbZz`-gp8vT#41C#b12XAs;vKT|_%OvV;~a<76XPSay&7Qvu+NtY8Ll zxMI4EWbD-Az_y@FR%!!&O$U>p?rx7f`bq+;*ueK0=bg;jkwr&aq9Cf+x zg>3+`?Bdc?iM$jzffPqX9j5GLFY7Y679W*(F&o|4whE_TivslTZ7m3&Dsx_63QGr!eE`EcVUz}NAYorua?$){l(Yy6<_L!~7*odCH>$bZk8Q*s}<9F`ke zQZb5UmS5{wkzlI}XD>pK?KBJw!NMD5oNexqrN&hHFQ~&s9lQqEdwPSs&o@UAo!zKR zKYh_axuvledtDf=+T0l#&g%WiRN80)_@YnO+C{@>+T*wo0ba%9h;`X{rvgg&#HZk; zIF>d?kfIBra1QnG@_!B09|XWmWrs&V9Yi%53nPh>Jq2^oINQh^oy;~x zMz{KNog4&9yO)x_s+QL2TsJjxnLk&<+GX`7iNU!-s(t9PC4UjsBFdS$bu{G6_LalU zlQ_rFU5B0_Yr2`!*S=w_wkpVA17CEqoXI-@l$%+}j;8m*>^_Ut6sL`^8G_H&=_2eF z=O5fB44z1}nSHkhTm25V;!U;_6OMb|>+dlpg`Ybd>~TJeI5$bNn3J)R$4~G3KOf~K z2{l-bx_p!a9DjtDx5d^_B5Z<3e+rr&m@9Y`C8FuM(J7%ZnPbSyiXl}G3&wY;ROHQ% zGtkzBQGv$hMuw*pu=rJO|Hf9;*0aKq;mJ7=6r$^n`jR}dknUEy7cDKJ5e;+@agFVM zE=h{-OODiZ8m#vGm!`U8{xk8R_ALQDk*zU-Mvz?r>3>g|9V61~%Y45baYit5vwWxO zyGW;j+%Q=sR~WpC5j>1B;q@yX*@&HIHYsN+%~p{4wwkU9abQqg2Ri??B)X^H!{%5v zBsz{0_k!Rufk-7fw68^jJDV74R)M?>8QCH2b(EdTCWrIV1y4KRmjKifoZmt zCC=PPYJW~hNR+X(qBkw8ZZFnY%(9a#Z=ZaqKqN$1A8Ac9ljz6klM805p*{8gaCmIL zB*c{jE7!a9Y}D$znYBwRZ7Xu*Sp*MXXz{;N;q#ALkw=WiEYK+|D5)A0hphJ9&E~~t z-ub9YUXYlT4x#;T39xyzx*Hxb)qcLyy8qnRWiAwgwMNBbN7_$@G>B7}R5$DkkW6Hc b4_|}I`4HNsPfPjd^_(_wiei0<)5lZC delta 223502 zcmce-D_$nd>v3na|8;KA-o@ zJX}54qiSD|(oz~dLSEG?Ks!b&@xLE6;z!bskuUWKu#8h=X->(GUCSi(CFX= z#cGBZyQ>*^Sf%|sjSi@-Qd2935RqJ4#g5Rkq-=YI<~vIDZ?XAqHEoO`O{CUdl63f+ zCX%I+N_drqsMMryJSFk)xF7JI&DnNscch8*4wFeBJLazS3s~}|Yrk&Y4lO$uG(E7e zS3a}4zZa`T<^&Y-!^11!f6eglaOFZ@sWeL2mhO?vtlCxqb?c-c$whA?w+c@#l8jkC ztJsmTDO6g;dTi4L-H|0z&S#Q*zaYKh=HSoD53H4flw%7?u(I_zGL?QWE0NC=UHDOC z%S%ePm20FOImziZa&4g?c34&hJx_YjJ+iVA`8+@5ntW;&wAi7yr z&LW1cli33w9erI^e8v;qpjDy(kwe6)mghxD3oX~6kpF<4dQdgk%waVWDP6)Fp>UQ%1i00 zc`7AxG+4A~g3Ob^MOYxa? zDpiG{lnP(1WwZAPyxXGeU~bV(N^ zlQmPnH_BU@UW75r8Je|+h5L>23_ublXV9zwR_Ld5-ycX#F9cAfE|A8skNx~;n?3BP zUlP3)$R7CZbgztwc9&-_XZ!p|&{L~eKtLLuxrz-6h^6nWVlx8Xrz>A&NufH=l2;{@ zF@wrO-(VSm$pKk^NG5}qH2pvZuRFy2#%{JaFqZzdhM5Dy>7Ps3p}E4EC0fcOw3X^tQ1}P<*Z4 zrae`+!?M1XR`pOM9I&1`8wvg>YXpcX;9yW-+u#{zl z$J0-yu}R_mXy+SjS9mgg^A&a}Jdyg2WV(n+0qiz0ZK$gYS>!5R%_#GfwZ3ds#F}wA zF49%$uTHXvt_qSjXpC7JUuxX`);!N4WsG z8s%=q>1{WCsHq#$1wCV+{EshN*;F?w$&Z?nIZ4zmmHEqw2q!s(J0P6s2v>Q@Z4r(~ zILcL;z5!!Ll(TrqO%ad5tZrDIjS?K>|g;g zYdk+k5}%{-K5SP^qPJbP;;*(Y))(} z_1nwV#EuV`3L@)69JGVXySB4{xctb$h_)8%E41D!m%Jk%ps8EEsp&m$wl1zvcXhsG zstM8Cbgt5l+fm4tDaiq^vBl3P(q_JFbn`iM#Z{I#ety=~zldqCr{4AngkaI=%n`SU z=?lE`p)*#M{V_=*vJJGc8gf}5lPqrXErcEKb&hxByXVbn#wU78Aj6v<>CF-+g!@bY zcU<&&C#2NpeAt5)3tdVR#-=ete0zHN6ZT^PCvTzHjMA5C6l`CPqd* zjTO#_X%&NNhvNOFMsF+gly>B~NXN8tJ2rP>Gp279MECu}%oC4}^1Dt{QLNw4W>TPRK>Ch){KjeLC-+yB@g`qmZZ(`G+?;R-w2CeFJoRJLfs%zO&H z;K?4eIYSGZu%u~oXunnL9sEpK&LXEzqPAuDiDmCie~iW~XJ@9HsW(qZWR)|z1w79% zR58z3F$6SDBadP1!3-VEGcw)G`7{~!Q*u6yWbFOq*no2AdPVe#PV~xKOIXUR$W3W1`JHpW(a$AI_JB7O- z`~t#aW>KM4D5e+hd2=_sl*9-b8d1!eF@fnD(X(`!6mC;S)yXc z?!~9*{8=oiNBH*WE<=cqw7A}ssd4V_|E=wCW%iLp|n}`N~!emCv~tv*!Q71}#ct&C|1Kvl;B@ zqNDVS@oZf&=J=~@S8+W3b|^brJc9PmXXi3bMESz#H{Pl1m_uPrRY~&FgH7aREZz35 zMmjcmTRV0#^C>#MYrLpdBK+~q8GhbcWo$j%*qe~+LhJ%Ys5bRB; zM7~$=s zoA&;4cC>F=-26A-9GrGfF*Ag3^rVdF>CO7V)_PQzSxhw zNKB*APRnGyt!K7$?VOvG9gBRrXW5`vveJi9${r4RA6Sc&NtXohOom4O-bh>;}^>PI{EXN7~(XvziV zQyNb<6br4LQ`(K*Ue4B)#s(A_9m`ax7?*R5tZ?vrI?~7n$vV$0q~;uy#VS;^s#5o-DgqjL>1uR%X?WxyBJ zR}C!R6d4G^KG`~YC7;cIEltxDt=j$C zP>0H5clOTeUb=DI(SQeF2J7v?xBJI(W*MA(fa{%<$B3 z-MuxEX(pPPexpERgz#hD{hvg&# z3nWudUuingO4%h{({0jN;20%r&H8xyeLt2zI)jERVQyu+G8Q0XFZ3d84B_O7A!K** zo_;_~d$?_3pPh@~f$`bdW~5}=jh674H((@tP&Pt40%U%~q*lrX(@++j^BDboG8>eW zMgt_aEGL}yyTIPbNo#us6r-i+?{kFl9Vm;X98|;Vh^Yl0;9^0-x=fZ*%jyX89UBv* zAfAu&78i)=4rxd7VFqZ;8=-7AgWaUh z+5XCufKw$6ajoAA1g>?uK;T-(3NS03a4qF!2SBoL+mo@CNNo(eG4oQuZoZi%u$v79 z0=p?L5!g*BFVqNjvrH7K$8LsVbq01*P{2!t%;u?1*v$hpck6w^E_**gM;8e2W_W>v zHxAzP6Xfc_8-I`kc+<_EsTDjC6})MU(E}u{MS%csXtM%#;kjkHwUhqsV14oucd#^v z9}qUC`SUDvydUjs;a=9TwZAm-)$}g#sM^!# zf8*ZN(;s2+a`vV~8>+(P?e~cArbJk3CVzmi<9)B=9rHrul$M|x>ugX!o3QdCfuuEi`c4}`Shzr!o3SQe&KkHiCO6i;KGK5o^5uGSNuf4TI_*M7M*85d=@Xr0J_jB0d+3|+I zY>*2cuDrNk1-ZljiI6WKj{8+2T-8!OhcNf69O;)4=6;n38zbd22y?&65k8NwvtK3R z^-=OM#5?&_to0m^O%=L72N#j~ zml3XasWc~Tg#DA!iGE>Y^Ix14Zbm9!ULql64&XJ#FZZdkYuSSrN6{9~ahHlkce%=* zo1f$J?rd&8sQ~eY+1#t5^KIiyBep?YezaHuYS4~J?z{iuc= zT2M&;o&g`KCAGUxlm8O#lhz>Ir#?p2eOfyX)sADo8}#(&pM(q5Vye()fp~RGv-&)lz#pL+@>_TGkf?5&j-&uXsZW z+@TqP*4!r6ry_IH^1Gw2AL$~_z8)QbM>!tx&d$_(1B5f> zwfslbnL2nKlV)H#ojr{$ezhO9q`{j?WcOb6qt&N`M>SyPLyu}>f$*pvo90-jRF5iV zn((MJ(}YKLe=52a9@U|F@Tiuknpt(Grd(q^s*^QtsF6EU2d9d~<`d-Si_K!>Up0xg zxxvn?iVbKYY#)v6hF|VW`4$LYY6UW@OQc$L(Ny6})lCw<)Pec%rQ%s1QV0>&*9(>UR38i=JQ%4^_t4p z$tYLOGrXuu~*mkF~(xJ)P44A*UeGIF>~M!U;&^o($srhF${rt#mw zWnvWGlJA?J(B?ygx0E_pcuSop3vX%H6yYr;bQa#y8&ib0^vV?BEfq`=-qP$T|Mr&t zoGZK~Lz?iGKAR)FrIyIZy`^ZrU4|{O4DQ_8y(Kq*agBLPKlZNomTvdv-V!BK*oX}& zbl4kg<%U)Ccs`5Wm`11Ov(XzP>85;ExN#L_S6INNjM$>d@F9BJ_kT0%0q_FE=VVg( z@!4$7rqPk#&yh@XGBxr|4Xvn+kZje(5_G@L%#q|@a@qAw^L1ePn56B_Sq84(O6=6# z_=_#tJjx)FUBJpmmP-7`zJ{6rP)ii;j_f5!c(1AK_4h zGZAi(55|lu;&C`arHio76*9(zvlb0A!aBJ7H!*c^WJK5n1>4&5ajQ#IjMe=b^5JNM zN-(FTd_KxeW&um%mw$chWNU!12j@Qg3VZpl(Obu==k3 z5b=CB=5O=>pW{9MsZ(;$x5siN(-p+$xvTDU#2mKoo#gQL$o4&=*Q40$n16ili9<4` zNz83aKbkO}Wo;QjFLh+*Eot=fckIHJMd96$x1_CoTLU}4j-vo6lh~s2UbLnYJ5nCr zG#wS@jj_u?sbKhbB} zWn%gp5Q+sT+R+ni_4~_WSLUg^s~53!wIWigJ35yqzkPy;{y%t0SvQ!%Bm>A0Jd%S9 zn#{ilTb>g}3r+xKNcO}yuFSBslD#go$Vy#EKh!Y-k2AP%{IUQS0uCddTYn=!oP!Iq zF0sPOu$GlEFLjsWuu-x7iJGh2T7aE4b$gT!El9&bY$i z=?>qW;|kRiAi+k?g#3(njw{^H5xByM2?AHxF-zbI2Y8`IaD|UVp?X|l(R6_;yf;DM z3h(e#CtRT|T7LGTiYvT2LEs9DCpdWHxI%#-SC1=XfE;jz7wnl@(Qy-0T;Z7s0#_I@ zLB$mYO%S-kHzWRyD_nu~fQ?(*C3a=U9GdryfE2zt+z?W@4I{{T;}#RWQmSe#p8{6Q zH*Q6^ti603VaNN=j(6ny5sl)I!Y`o1A%#0u6;jB&WOuazDd;c34F#kyQ6pXD+qb^E z+tV*`Tx54Vz4f(#6uh4l+qcJmg&o((_H8;8Ki|I1=SGgx_O16cv3>j89zVMIYXLTN z9*>34j@h@tyJP#d6$cyqC9!>bY)=qf{j~rb{`$5d;2>HU-A?`;ASvIz72%v@nWNl} z_e+j<)SU>xp2-5hYZio6_fcxku* z9A19jVNIz!xbvSEfJ0aO0C4Dp)W!gZw$BSYA%Q11Vkg8Qp4$n3`l)t8=4fFjgo$Vu z`CPsL9Q>abfP>fb!UlBZiLe2W=CeVc45IUnuyvp8r+tsG(fi|^L598;z&}8SQII?b z8RGK=$dHLq4v^v0_=h0F{qX{1xX5EvkRcr=TaD8mZdX{2*u#DN838i*Q8cU~^v!)2uhU7djjmp|C}Syz&wP%EV=u6&Ix%`o;9Si>cNUVINX>Fau| zp*7w)*6`Z^_Sxt0hGqag6Yw~&28g^=(zlmG5XZ3w5svOI2O`X|21j~RggMqA!j=xQ z7s4ECaD;;pcE%b+d|5}CAl?aU=$b7Oq%QK^KUBa$gpD2K+X!>O!4Z~FkOK}5Lihe) z8^4G(TtGGsI5^1sgRnE;AmXc%FK zM}#@xAi~i}^7nt(0S6I*N{%6eGvFZN%X-M4Af5vbB8;}}MVJE)j__v)bHG8R(@oxn zFb5nQ;e7~mz(IsTZx_N&fP)K$BL^I6{t$q}&q(D<&fp^|;4l!s9B_E~cL6wz7$Wv? zD-UB2_m{EU%}~*X6JrJ1@a`W#8{T2x9Kq?hF9qUYd`=(^zx5M{!?LjgacFvgZ{hlg z(I&MPTeyFn6X-+a7_oyp3U-}i2lq`l6xhK{oh%@Tc~evf;+sKi|Iv-K$sqwhjK~%6 z!-_!yerTV=8B^hhmN^1`kbVdJur54?Z{U_wISvXkHz(~O0x{x%*udR!Je%4E3KU{P zhCm_wCkqs!qK!ZyTJ~czPM(Om412aQ6k;-*1HOTq`~6ciW5M?Q-(v&!r_lm}I5`?c z071N&$8@JhQ|TK4L1YXT5QKt62L$m7e@~KjXkp@LXZK`ne@Lab4vJmd*MDe2e>^C5 zZQ%@8@?F~-NL?Ybq#qOM-h%>xXn#oT+CGJ4$+2r|IwfF;_E_tkVTfZ`7y(1LqC&^6 z?cHZp7(!H@-b3u#ntlqR&mR;}#NlTJ6!95yIiQGT6WHi8X>lU}JUt3SOj2DXc{vX8 z@?G1H&J3gZf3&%VJ8BR_RAj7lIIMycB)6aTFfD9d< z5gh~xFff-WD zcWr$y4cGk$P2_+ea!`ox+SVNs5X4&_2?#>@NbK6CLW+FXw*Tc%=5buo?IK(?6|BXY0J|}R9%6Nf8+!!Zth>^(1aR`Y! z^}r#7%412SF*w5A5O#(bL_F+)1jKWQ!5;1^CnLfcV^9h7kfRXK zF$OhbH@OwU9Aj{#H$TJZtwHqFGc4;?l3OuSlYe51Z>42ifD@EA!j3Uy)w}Q(#B+=x z0-eMGgKivS$k50S2HP7LXPPi)fNEb23q z?Y|pOM|WWt?yjP*4`cK1jiMjyW2f(>(7@r$=YAqx9L3V^7gB3B+jqa0&#E7Ysaik? zVdks*na_j8vD>ok5CYT)hY+;rYOeGT?eAXK zfha$y;E*sMis^v6UO@ajJkD6c%^y@O;WFa6VK)!8a4bPN4J<*|g~0NUY3?d(IGixj zRgQ-Ve@N>2FF0Y*o5aL#)zksItI~`)t$hD1dCaA2Hhl6bZxX6B9Zg=Mf!WH&(WHlq zhDS#qNA$hXq?OC{VF;GZqHnq!83w0BKby95`Gm){b0a~!{c^|@;vRzfZ=Z8jipP?o z(4B`RaZ`Hnf%gR_v;+jPv+fih}5&YDs?l^>w;Uh&$ z@>j7?ipmB`0>h1X9mgB`2$dh2Dn}`arE{~C(a#Z`qScTf&nb4+Zl6-tz2Kr#M!JyZ zp@qF6-{UxY0F0s)1eF&nrE(Wu(Yul$mm$27#u!M{_$lo}yJRU>Tu3-=pQZG;Ln6Wi z!9X(sI>Dt8FT%m*ZKsr^KpL*R8A`LMHcL6;3RdcdDpy=lw##6}(~S)BXbw$P161yg z;_pTxI{!SB_f4>QJQvF&{lfe4et%+OIB>Y0O12J!kA^_=^Kske z_$omr$lM=ftPle)%8&9K>Zf-76NtZgQt@#oNwhpw>ERBJeA1LrJpB4lWtlsPrz?jl zAG+hZmT{= z+K1bpXpdt2g+{ylj;nFXv<@`XC1V5L#I?7dv?Ro3^Ey83JV`WNyH3%0pubkFQ_}eF z(sjyc{%c&P9Q7ozUUQL?GoW43Pl@s%VNv;r$T>&lcIcs5i~83Z7J1cn6g?DVZh;<} zTB-@AcZVnsJjgiO$4td4N&-<%??BOY77*WUM36e!qx87G%DG$~QvyD3v2URqJo1G?4|nfet4$Qkn{)+S=D z?N{w1r{kSd?4T%#0c2y!L9l)ac8t_qUE3^pk4Aspf{3d#U1W;*eaORSYBl=g9Qu1z zUxfyuPi!eld?1OB9t(V#n+RO(TTtdpA(ue&0F=9nG-b#Ds#6vrD?O5_m;)h#Io%W* zMD}OVN{Ao+f_AK@EFJi?wyLIXrygM`$5Y;kv;~=%WndkUm|Oy_8D-O7 @)-VK0O zlq>3&YL%pp(KU6QwnN)*`&J1ECTTP;QyCNtmALj>Wl}Ji-)r79DCuIYY{g^IG?6bL ze7D$Dwg#r#0B~I+IBsnXNw-GZLpXkV7@`~aeLRtqLLoU_TRV~&p1}gnRNK5Z%3P!@k zJ)6PbDTN^<#WGjKRvPP|}b7GEzAkN}_@f1jrXNz2u7pXqmy@vLYoQ z4ANO{RMNsoqDLp_eM_z+uT@?MBTxIRg^yLsWn1hnuO6wK4kK~?ILCYw{bM{li0OCH7m^JmHZ7l52MjK90y@zeL2MVJh^?+Q5_2&T;t+00Y~^v} zxaFW$o?yVzhdrQ}*F9w`!j?Em-h{Lam_(yuF$53x5w=?nY&V|&Hdt)A*x(^$2ZD*i zu`n`;H2tFc`tTc@w5f&~bOrAc0F50*3m^B8&!eTeRUQvpdJe7poEo#-WB%RJPtZ~( zNu3=P9e}FfOiBaOZTIxhjOg+hjE^A6c!G~Jh!bXfCPbV8QKu_i1Br*h=r#x~48b!P zPbi)Nc!Kc^#G}L04^JST{&<4$^ugngr!Ss>tF& zVCq-Q7g0vCIU;6$q$gL@bl;_N&gEU=D(%b$118AZ2zeV_r5y@d^r*bC?@u|AdapC?H>F4nrId+y~MkVq40-OGr7pWhVtVI@o`8r6Ax#ASIG6(dg$jK_IRb+ zK@FOqfo6k-(_^4;6)QoeH=yliWk_-kP5;U0ChZ77GceDup2nO$Y@gFHm}()I(|P0S z=k#s6jQijUR)gvMqnB(&S2<+-9Hz5Im9e3LjQ4^)1u}M2!sj!WudOE>HdGdeW(mTQ zm#Qa9>IrAK7_(ji;lHEA;Aud26X^2t5h%~4sKaRB%T}`^Y1kygaA>R*S`3HT0qJ|4 z8REj_44nc)iX~msRQo-aC~oK~Lk78q4ygO8eyZk;lTyD#C3g_#5E_r;Q6Lw_wk?I~?=JTb0lMY5tH#pr%D2ogHsg{y$JJekn)SVvc?p-kVG}K)|riYxTTx?4s3_FU$ zx>S7Er9#`7W;;k^%RzPZ;+pNvNN%X4*DD~TXMtA}6kDD~^^>Zb<3D0-84YYlplF2=E0zVeMm07C&LMp-l+MP;^u6cCNPGLZNdD!KWN+Cbip{A|gSk~E43#nZ zy=B2QT+!i&K$x6Z}y zO!OyrX1Oc$w0}jNdy`aaF_Fu-bp=^a5YDJ_mG8ksiSd(6cW}PTT8x3gzruvUHs@Ad zM+?eUgE{_{ueE5vJAC;>+1XCAo?&=Y*e^B|c5?{({buz;-SSVJm_pB?+yW4E7We^p zD_q{2x)W0Y(zLuW5Z@S$?s6P%s`9}n7zc{Z8EXQg%rKCTx`?XoWt!F9teHga%dY_r z?J1XU=~7WH;DCd*SPuE>X09?OtiNY?uh4`g4F;@{yJhH0UA#e0_;;Cz*Uf8;a#vjB11)%Y_lz z&5A^B-g3>tt|#~e)AOpR=q)f>toQ*-=o2L}F8|$Sy0DW@n9e~}MkS-O&QYuQkgadFaM)l`$ z@fmh{t_yT8K9qqG%$LbSP0>TSk2`wEkZnPR1?VDfz&I+|Fi5qxKL!KV$?#*a`dFsc zZ|5>F9p8vN9F;l5@vc*4daPEwuCb^q3T3V6IrN&YixooErh3Z>kYV|jl!^vIzUuR^ z1)L~fJytsr6*30V0U%livGl{Q6}>1H3pb7YYFq5R99!uU4Tmep{6U5KeY0JMRQIE? zK0JK0t`k1KcEZD;XWa}QYgc2zEvYL!iE~LiFAkGz)*FQ%@^3X+F+7I_7VVt_WVk=N zV=bEFINq;E*z%-gtHO}J*IBZy_mT|PZVZ-e2T|yBfCOxdkK-Wu#*=!OMS)mm@J%Y% zPI>s210UyNrkA;4Xh&eC>&ypXUi76>P#uVspdu@XgQ%dn=zd+c6?7KB*YxnK4DcY* zUfzJ!Hx`2Ng(nXaX{Yrc!$QGz9xg}dsI(&r97bSUk8@1$8^k&8E!ohYJMQ6*2tP}$ zsvGx*Ol$-btvC}%*P@IS`8gB!dP=rT&^;Y19lz0AZBRA7rpOz09`lSh>V{#k-o~|q z6&Y^eU2u!p4sLBXyTVYMAngr{k#_V!`r&A4$0c04c*YfLT%n%vHBtN8vNhJWl5M%Y zRTm+K%TbceKs+u>Z|Lvnu|im9Vv3N!%bU{FImj3H>b80tMQ<4PWQj>y%i+T6?v6=X z)y|S2{WUX~MRso!q(-r|P)VRnMaqakuaZEebOYXqC{5$`O8-~D&a*fMztAcT` zNh9;&e5a? zYuS?2t=%vP4QPoaN^j#gVrJvLti80O1~w8ORK*xjd~kC3hHtWjVIUo*LCZN)_xek? z!ia&0-_%)NAPk_m`8?PGgB%))r8cRFCoV-ZSt?M zwuX7JC7_}ly7$H=$Q9bdx<#_p<@%*sA%A@_Y$b?Cor)fECm-ldP5NGb*uxS5Vdv}^BT;jg2?mv6@_Yd(WXW6(6cOcSK!>^#Wz`$>6wg6irz z9`K1+UO+vU_nE8e)=TcRLVr*rkHRr5EeLwun|HFW@`)Z64s3V>#{VYthZnw(!c)23 z=#efRM?0@w)GS?dN#yaoJT6+wz7(jp^}&eg=!)iH6a~%eT5Gk_I2R>lN4w}hv9|12 z5k0T#$1~l~FHmI!yGK^^SNY|-9sITt?{_}Zt~`N{V~)g-`TOP@pFvmp{Y%yT!w5 zjq@gcoHH-s;{*)pC}@cT__!WthA+0af}sq&&|;s43~tDj1cT8qBUAa@&@m3Jhrnu;EPh=$``XcqW?4sA0N^3MLKZby-mwlU-*@)S-`SzAi7 zbnUxtWt!4%B@!&@YaQk7(1qLMQdB(|!<~AqiaVyc*emi9oT%oV5mUv{h3l@@DV3NV#W?YrvrV-TF1s+asORp33R{VW(#R!B(gQhOX^8e)EuM{HDIL zv1qO8YlN7E3l8OdPUC5A4uZ`B*@Bx$^e^ul!DU4sZwXe>AP;oF1 zcJKp@j+OXrYq>7#lk(y?V!i-GPSTAhi#^p6$Vk9<8;a*=y-5+HXT z-6MuEuN&{+snYK7Rp#mgWk+$&lMB>bQzxEs_=H8d6W5f;{zRt@K)b9kACz`}B-k}q zlKmf3M)W5M9V+eqQDyGq_=q5{DNgOV-$C(qXg$+acL7%*ed{&l!~SH|DD6Q^KC3$U zO5nhAr(STVN}@6sM*y7ZaSp4T=viO{ae8ymvdt)~y2`=kp*NLv14v}nZU8-7_+{lApU;r@V=@4R{(N?(2|)?}(JSHr%m_c9(Wc>mqg~E1h?tIT)_P407JWK%48pkcP@|&n5 z<}6j<*&m$Bidl-Xub`|Ylgi8S%T;*^)$dixhXT|6ADH;miI5(P3uj_B z2-vZqdeL6>Z^#5DVx?1A2NUB_c005>2M72bZY;*B?@NGixmC}XLEb}S7P-YJv$Duu z!}Lz_ML??;<2<;@Q>ooxH+lGRiVz$m!2uF0n&dcG!Xa1A;lkO%^by>N(^lx3^GM@L zY#JYM_HNbR!n8$xZ*axcz=}x)dnQ6`ci~+1B%zm;_lA+|va)}Et^Efm&Cy-X(??h>4dNUO)9A6 zKB5E~4k)}P9p{x&l)`M1k=QC0E?$VVEw4S#JMUw@j&fQ!QlHI*N6-r~Mjv@mbLCPt zX~Dn3?;~$+uIPr7)Bs$w_k!^Q9~_)!Lv=1t#ttXJ2CZBTC*>Tz`hUKwe(xXbiG&5P zuokDqM-T$EE>)y)Y1+!~c`v!0arkskQx{78?Re9;S?&CjrKK@r*>aX>- z4Ph7?->FA=zQQ2QamO(pcekxM?l{Kd?vCHb@#}(LoZ@kJ#V=0rxKsS%6pyOzEtcnSYIW3tDz?J8Y)+H zl}zXGUhSK)SAUSoBf2X4MqqPH2gC!>)(M)5JDz&mNPtb_AUpm!1nyCf#5+G{#)03x z!SJkML2`vqhhO!+2*2nS4r?*g{1}u%tpT*=!nuQdIeUB;LZ3@i8S9(KVa?G`1+XTA zLziEIvuEI#fM*M-A`IIRT5LzaSHUgKTP$Q4>tk_jz?oroCtO8D}wymJx2VA$P#_<=(lgAzxe z!lkGAsmM}(D$)vxecc@g@_q*Q?$Y#X7$*&uWqzy)m2jz_gTz0FHt49UpLY&za0wqY zY_#`!3(jwEe#=l-nAo{gFKDGj|02rSzWCvC&Q`4b~ayy4?G$#em; zH4o`gn57liSrfH8^w^$;x`->&`>(e=#|w^krcyry=1x#rnu{`{Q~EnG0q6a88>g=Ls}faALekVu{*#&RND+nfG%)W${^-Y%H?qY@4?i2Go5q-!hCRS zLK;^I>l+yy-N1|$74eOpzWkUg2Fhl{y@6gmS*UA zBFA#W%x*~YLRvJoqa>U~JHLULT7n4QE*E*Lypb2v-Aix#@qlWzmBDd(43%c0oTC?S zW0l}e6}CHU<*>c@ukcP#hk?U49nh7Xc>!=bADnWpYNwl|f{8FSTJbc9;fqEV!R9FY zI)(B>&+YlUPT}5|P>66!911ZEaU96!Gr(hKiT z@GuILI~CZ0w4R?S`Ogwv4n@U0otqX8g5^l>V5jpq8p6{}PU&SxZ;kYJJP%fJju);) zTs*?g$H{q*E2Rbv{vFXc60MEK;s9qhPdPP?bO^>-Efc0wM{Wu%-;Vcjk~~mpmP?}O zxfM!}ToOy3QpV=uKF5bW)sy_ee!lA67;{gqb%V^ElnuG0E8VAN@6!HN9*rwB_G~Nc z*+MV?4neSA;Cb{)g*!I4m3D1tbFxNx_Bqnld*uq1b}ibnMS0^nQcU_P&7LPAVVNM; z*nd&#t_*yh^dxPRSCGrIC35lqp>(y6`ahJS5ElPKiG(S?A_EQV$v@CSycO^9BxI1g z$WG+1FV*3?i;LW{3o2>kKPcVr!O6PFhp-#?U;jbrW)J=!lwz(RD{O!DN}$t!P_l9o zu^M@sQZb%%CSNGk<4N1_ZxG8ntu5gd5P7AYkxQMG#0j{_vriBuJCx}YNQ7G&dgWr3 z64{=PCPm7%2_(~{7zCKUHmJoyP9MAS>pP=<)O#3qo=2#$!9RJLrA*I5ALRGow-WvH zLzyiMs^GL*ld($s7f2lOP^ytVGON3K(MAl`ChC|TXy@(6 zDy8r*5>6f$C7S)UR22`gXX|Xw_8DkTSEj!}Mw6?`u@^}5W|CN({p17tiTE!*>DXK0 z|D%bY{&_?3&nImI2K`W{eClc6s4bA>Q>&D$e9~itC@T5MYfzN?so3DfcTIv+O^WaS z7hw@ALmKyI^V|f!CFh-%Dkg&oj+}6sRopuO%@0>`w7Nw`_xUvZ{4+R0~Q#ic1oe`=GnM%5%H7lk%wI&PIa&XDHHHpf%Q%P(8!IhHf zH%Du_O7aB7V;bqu-O-q~DA`y;QtS;8r=l?M%bv#-b+I2GXiiq@z3x?V`G88hkz0O&5%{f-_yuw<+~S1VrY(^eC`uyNR{k>O21hok>n^7=a4v`tNeBc2EZ){WAi77jI=Lt4}Gy_HFGaGUK#u+s7qeo0uVoI_kg zKM$6FM{8a(2FT}ORfunx!;W32d^?A<^*^4*)n&b2*r#~UC4&?9^5}8(x9c|cwqL0` zC&@qA2Yq)`^w-DAn&-$Ox-w1thdxrG#JmW-xF}8ihkP^A)PKk~MbzRYcY0hbCgeF~ z8_za6P1Qalm1-W$N)!K$&wqf(O=Q0d>RK{IiJ3=2JTgQyV09&Z9`2~5riuTv*8#DO ztrBf`ZmbeDxFyB^&z@&*{(ttu{1MH^`bj8bzD%6_L#?B&5DVGo>I_<XO4@6+syHIAb>?(g5~ zqU0CCzxbz%a_1=;N&oDkyi-Wh$YiCukVK^(#^>|-H9j#nLv`>O0em(Rmi+X#SJBq{ z1)t&!kR-2AQVpbOm-G-hnpAa{h_O2_^%XQtLQx}e2Of!2{Fl%`<>OrHr&Joq5HejM zM$(pyQ`#Fzhw$O(DGC4bjGKd*tGUDFi)j;VKK_hSXe8my`XK#0rc|yJD3@VEoIB6| zrA(kX2DbBPWv3ASZOZ2zFt6<=Gc* z`zu=)kq%sDi%6XI%OyBRsR@(CX8@GwV&aEE(7u=i(tCTAUd1Fit|zLxvKQZA#8Ztb zV=RHPWRb)nDiN*09gcv8y4=Dbwi^#6 zD8{vJppx(s>8dn;nKX^Mj~?QVDIW)-j6e%EgyS1o2f|7h}ECh)ng2)TJoB7L$~I*Kn~=5VJG*w+IKB zUj#AEJa6ME9Qfo7ifg1n)%N^T_K91!O6BuZ>-Q@nC1`4n)t$t@U}et|66C!JDH7%yr~QWV z(-P9vYq^7_B~Xbf0nKJg_Y%-7uoH1po-HA5!(OllYtf7?dx+^BJbZSxSKclm(Zl!X za2<}XG9KoQm<}%$VrT;=JirgPe52zl(Xp4xn!;%YsGN(bgFNwbiAkhe?dj3Y zK{v&)l*B|HpUNy8GjB>o-%Y9S>o?Hrhau$DgdD<)k`%I zipz4+-22+-`m|!D({fl5HM^D3%Sm#RRCJYcf}ZUxB@FPSUJ z4&rhwCOT2M`zmQcCMq$duq=+n7!#FYrHB}&%qt~p(!@U-aQOI_cM{V%e3~P}m?#tj zS7jFv#|J6Dc3ceBT_Agl#@M~JGEj!{i`ud{KLYjh&$|H zDyN=-4$kpl*17hy-dM%Aj0A^>MErXQo;doE#P;e~Q%ep*;zJgt zSi#~(#Hk~QkG??jy4}OeNFW*C4F`M`3}(j(%Jv9LRK0V58~L{5eGWD=k`eF7(@M>= z6ZA?msuE-lbaQxd@3x2a;$PD8aj*Wz#O>RZqZ{yb5)VNm&}>p#Y=jeBrS#oMS{qt% zbQwpA`1z(*K)LY60NI0sjp^-V>lXJ43u=*~Tg%tp!x63K`U+1!{odB7@f$d&D`J|- zm$!RWyn+p#zb8nxzma!qb)x+D2i_Hb1FSlVJV(biMcAj}$_Ks`b=$QSyqt0Cf$o;# zrQNJsJbfxIw+>Gz>y5Laaa8_FA)82R*R1XMn2^$b6QG0BW~JXolK6jS^Zoxn*j%i5 zZzio<`hv~<=yd1P-CFe%iBR=Im7{B0D`Pg37{E?Nn*mQHCo=t!g;BVloM5#u39*kc0~kGDxwdUMpoKbdh0BDMY?Rq*o>d<}{ir@F*9@tKD% z5M(F-T`e%ws9>BVB(BPD!+4s35wgVbStM3@v_Y5wb=FFKhXU?2Cb0$pp0;^Ac)l_r{*zPi!b{PtWRULT4mb#Utc)coj3lOy}Xy;ZI zqqV^_qBC1rx-nMiiTY~QRTh;kn|z;E*J-F4#yiK2fsRK+`Z;fqGpemfhWwUb25-_FuoDXdK`^m8cc zT0XV{8?}7MKD6>Mowg{65_jIF&Zs0*d=pGE^vCMmuEE`c$`LSj06+h)03o{F0VbB9qMV46D{9f$If3LY@*V{>NfEWE_-qtidV{rwxjp|47G~61R;yMSATL^x>e~`s|pM!b9O7pn6<8Mx76cn72`jI)7QIMtA-$$7bMV+z)ccO zYAH1hxe>0VZmtEPPAM*tC`#&al=~*L$6ja(pKTcXrAOXwb$lrmsURiTE0HH9`t<$)Hud1_~-ZhX0I9PkXH&DY}?b<@mJ75OOrMDa`sM-BvZfjXV zI7|I~F`PQ2WOiyGYzWiha;J-gRXGY;)Vu%!?i zeY=-6_Nu2wK^`Xi?YrG8!c-1-Ti$ZC~K9*|vEY4K$ z9!8z=x3b_yAL8plrysK91G&-nhD*I1`@xtK0$~G;@$a!8o5ENDeA48}oWcn_Q( z1FWPXq0%6)rL2}3qD$66BB|ys14gaox!llL?ij1`X&KG*W`J=i#iOVW;8s_A(#bMb zPtI}KpZf0rkW?&GgD09$#zD3C)PHT!7>62)u}QT22u9%}_fZI; zsv|75(_Mz5=YMUc*1%*&$%=OwnE3w+u=X%%sQCYPnD~Ef{C^G8S1;Xtn7Y5qnngbM zF-{?c2^*}^juz9s8!a$ZT1<#QA#K^47)^SQ^^lRcTkHL{z|bjK2rq&h#zEh^EGQ}? zQJ(?^kjC#VnJ|XC8OIfXoOTp*3`%({bv?$iyiYgRj{w5binjE^G1kP}nrM7yr_#w| zEHvpaSAH8^IuBfgtJ5ZFzK?a5t4OUmNPZj5)}fH&tR<}HIqhl32^JWem#Duat>;-u z`cbrVw4$B9Sz69_B|_mc9?)Kjde73TTiiLtorVVORN4x1dnuC^;JilR(-x} zDh0lWV6ESO^uNj0K#}d)&9})`kILS|VbxcS>Cs?*A3xaGh()}+u@Q^-rzj&9@t0BV zv54#Gjir<*AMY9IuXKyq~py8I2?mV&<8jUEu!Z?V9`F=&eE4KT+b1r zLj^VXYv!GCNWYjx2g7-wCvIPr_bLmcufloActz+A@y-EiZ4fX!Sr<5dcqQ6&N*z5w z3Jjjf3f5q7ynIc1(UQm;s$Pc3nNB+j91D;E+E#;Xn>QnI@2s}P%h zyHmL`oNV=Bgw>hO?xdIIngl6MCT^n(?e-YRo}K|h@*J!-Z6p^1 zA#an3(jvJR<(-6^M%{lFib5k(>G_i^G_1T=Wul5b`yQy*Yg5OX*t2ER>7$b@rg@G} z&EeR8oklFi(X$-K&zj#$Q1kNlm~H&zCVK9B78=;E2QF2sO@V*0%VX^8HT2&1Y$U5k zjemgk+e~;6D@Chn2%kpZPk68=W+=kCc2Y-2y~^-m$8#T2OW9s%X#XO3v34WK3L)1W8 zc!34k8!s`g5OTng7)m25bE~?Kc3)t*5s<$y3U|_HZGcdL1O0Y5Ww_F&(twLNFTRgv zTx2npJ?PdXbYCjb4Qby+)|}0vix+|RM2Ys2+h|XtkRO5eJX-!E(C(_xj(QmQNQW(n zb}Kiu+d0u*`XoJifd$%EEdD3nlicdsDB==(pq&fH&(|Ax;|fv@-X9@NI!S>kt@6ZL z_NyN9<@c8_(1({;6Av4DbMgiH;}UDipNOD?@D1dNQ)%OuA`p9noXgDLdoH%X@?`$+ zLgoF(ry9i~8%#BHMAr%a7-zW)k~l9gS{-R_M<-Tre{p6bqi-V3A@tV@L^ZZh*-sFz zhbiMHmPBt|fpKmJ0)StEL+4kI5}x^N$=1od{%kSnKf!@#yEAo)ObsL7pYcdRE-n0- zCHl7Ci1W`U(u>YvcheTXX+agU+jg~-5+r|VVVa!iB}XANzmJv-YAKddqnO^RVkw?q zAS-nVryX0lox-oOPa436kl(q&#V7v7ziy<1M-bmW{1+A+QTl@6(eZ#uT1>%6a7vY% zs(nbGLIZwbkl_s^$+GiA32j`fa4`NEq|WA zKEQ`k-CtRckO!S@NC5kU0e>#iqG-miY*wF}3ys4p4LvoP(r>2DVOIG!7ROJMO~+Cu zo61wl?qgOR=_Pl`^P0=7?;-sn{!!_5bZQaIV)m-mPI75AO2CUsOpUq5BEyFQ;C-kd zfu55wx6z=Z~Pc%UF zP-PoA%T}0;ve_PbbcWkN0fYPrcy0}K@2EHWu5BdLjIdlQ7NcSSD#oF4sq3f)H#jTW zG1nJc? z8~?@r#_IaNjiPU0Cp{mBmLG#Ib9oTWLMZfrR2WgNftPkSFkNNUFlU2HAE!CLv9OE- zAiF!r7eH?JahBuw16nZSuY3Af)UzQi&2i#4Ml8hONweN>tO3u(qrGV14VKByQ`rqR zH`N4=Qcf+MP5J_VIPgG_V$DoS2OO_< z(wsloBpaccdqn#N04v3aj`VbZQV%6i@Sm&?8%h)ZWRAe!Ag^3VyxqWvEOm7~wfqYn zkZn-w`}HcGKllEN*8Rl>Hup!Gt9&+mtmGnn&<)0ufFWD-Lb*NbRSbRVbBndrX1(JK znNJ&TF`Hk`JJR7Zp!f{1T7mjc(urH_JkMIM!rs&AznQQ3Y24e4;qoZ=*wO}%}X)U=E9mVyxe9NgQeOGj+qGeDw zEsI)Eq?rddtg_q8x@4qt5F){&nOyW!q@c>cfplFOV&(%`32ia+co;u7nH-;+c~>6V z4IMGeMNB{XwxkCtAg$^dYGdK+SX(+_;e+68W%uIkv?_bLaE$fhaAJ{&xQS0^?<#ndYZMgA%)iAkKLsMLuij5pQV+%dch;^r}yge zrmO+YX~3I|EU(p#^3K_JbfdhpTQ^#+yQ3Q=&Tf=DyHW1mjWu+&0dJ25u$ezU$@)2N z`ST}PZb>hDn$j4HrN)@!SEDg59OWjPA4@N~IT~*#9F}h+?AGPq_vlTZ;ib3O^Tkk*OwCevi1f6O6%63{~~~| z9K|SdzQE<`ARc~Ci5{LC(pSD$3CJu{vkV!E_QFwN2rVxxplIowfb!Ts#vQ8U%|jKg#& z3>O~uhw>`xzh&QIZRo9V9;%h}T9hJpx#8I*_5&$S2Jm+BjrTvcxNAn2IJyt=-n81Pg{d=vFkM$w~pHND&~EytwAMF(S6se8X4|F zpl;2Ea~#UaU9W*)s;Uc4p_mVukHOg;+B$n%M{a?`b1|R(1+97i8!gFOazCt(xN|?O znxCEOq>pzEdXj17epuz-kj$NHsJ3zmREd{w_t&)_ZZ+ceepr=R>LRX>K`-3vt-T)( z#eli*hgCgW>K*%GmFE3H4VIJ`$-~1+>?> zF1PFTmFnS+=OTGi;gj$%!xAA+F|5T~=~NVNW6iI{viW?*VMkarPv-1l8qk=B@jvF! z%*MQ--;Zmgb7VmAJ#aYv3%kR0j^`Wm6y{U!f!ja>sj>-gXw7Gs2c~fQOnY55FBDM! z81BV*U?nw-b@&urf+N^dDEDDmNBxomj$jVM5o`t0vgaJ8cVl@*NFCt6FW;EE z*ZLz+rgQ#Ypdr3uy&clFWT`1Yuas+jEBM}Gy|cJWR(U;dC~h z?`JXeYyyA4%L^%cR zFMQM<+MdLxnmtkDO-DovK9`w`0BAU^ZOKP@==hobl)6FZzzw=aEU;>oQ@4UHimRms z@^tjlDs^d&SFWO`rm}YQT5E3cu%YV7RqDzd*^>Ei^UqJQf{j#`!mUr^eR_I1nZIn_ zkF=>WEr#Z`;Wt8}oY1ABlkdF!G6QA8Wf^~ZCWSXL--B--2fp!mTsbAB@)&ECG!oD< zRxlpeAiPFCZcov)RQ^Q+9~72;f)(Qtk{18|dWeR$-I?LpGKk>cofPlIf{VJjQD^(Cvic9f_sg4XuDDZlg6TjgMe_#{zmU z-3_h6CE}Ds>tR}(!6!@xT1%uJ2huE+dYl_t$DcINT84CxMn!bk64SCs9>W}}ZiUE; z{x>8vd0tt)++cVaC9a2tx2%>7zG3l^rlYja8e*6xQch>|qCf5J%wy`VMGv|FdM5tK zsILvlJCjFHVk&Ri0JIHx9M6}dQW;*1kz+eP6Jxs?b=25qVr=nm;v>|r3(u5edq)T4 z*dkkwZ8X5keXp|v^i3768kcwB7Wnml(FFtS1DFTM+*MN2(V;8PV=X35VFd|)A!ac2 zDED_BN`I{$JPCL~VIEk7pyEZpRwq9Xfi{k=bmM7KOYF|avZ=JHJ8ubYDecZr;>MaS zQ2&+I_TVo>ygQi{JoV>4;o1;_=r|4N$^8S4JO+ASHzLPuK{Oc|;rS3f)RTurq@O@I zRqP8Tsq4zyBN)JMLGm@=Lfx^mCnO0UnN0`#^4^X)eRyxi_B-C{%U3fNKz;l3-sVXt z80dJuKYxR<_i4ZY-kqO+f}R@yW-oE<8NhompYO*?Q7vz42Nb_S1N!qwN*IK$mC*D- zJlJn`Nj3e3^JVpb1p+^=qqT$htdM2%jF((o=iG;O7}#~Ah{3!&TR{^CW8C}DTZ8$# ztS>z_gvYQ<+BO6Vdx+yZf5TZ;GKa2<=LkKkv5-s+90U$6*F z_V8a3fGCR!@ZvxkJCX+kx9Z{qJnpttAP;F0FcpsE(IDHAk^C#hN;Vgc=A3P(P4{qz z_5hlotb2GCybrdGMM7(tko1z-M$5AKQd}GM$>E3CmvkbBw_=|;@JwJG2AiwTcpm1} zZ5Xq94rSIR)M7lhahnGX8qZrcUF3PkZLBF!8RHPj(EWy%N`qBz$9SFtQQzw1uKPVClVYPys~(FFN%esRssnEqOc9T$4ky_ZkOelL$?Z&Aj*Jkhdq zF%;NEvSad*w5RUnBTzIuj4s{F?YMx+Ci5V#-_^xeYcVFmB%W!WgDN(fKZ#F`DzT?W zKs&O+;%&S*R4tggG=l480~r?Q(Go%=jI;L1JkcW+H814R(8+wFRuaFdAss@z+PDuq z^zm{-biqS^9DH)r7O79dI$itP$dzvN_hjDqcEyC+6$>MkLVCB#AJLR4ylqyxMM}bw zr8b+4p6h-@Nm|%*K8Qhp{$v%PT{d3tv&ydEA61y zyV1ca)`XI7v2dCgf%0gB=~F@wyDR$$L)cU)IA_5jgZ? zmP+S(Z`=dt-OWFv#ztB=gQxK;kJ7;zd`?4LT#*L_(*4N4_+2O@>!6rOtEn{9)}g*L zAtn&PU?v}l(JZvn?=yK8VBxvKgev0v zL^-3kvmzOVc*_#gDOk6y-k^&tha;2nxT{7zRUT~TKbtpZqbPMYkMeb9I)TBmF>N*k zQg_Gl*?c(T+?&3g!}s!AxsHwZ@pBqGNQ)l8p0JqCKEPu<`k+gHE^v51$UkJ}&lj?S z#~o)Mg1lm@DdG{{-7*hJ(F?$}Rck=oWsmSKaCkWJ2>;3)jhv5Y+XB9WS3gYmEaYFa z^|)@yhc(-R@^Yi@g^hX~OHZ8&9&y2m8C7{h~DJPpDl2U>=`v;)1rh_{O^4c5ouyK`w%U7F174`V*SOm6BCVx;5V2lIdt z!qIFoPZMk|&3KFl)xCzT+afHQl7FdiDYtr!gvC{U4X167@yO&XEOGL#;IfAdwN@^7 zrB1>1G1_o9;;!k(tb_W(?oRA+{zh1#3k6$f&23tRk}Z@@KF%BF!l~O`BKHKkabF=2ECZ)5Mx@lk*~{N-u1oq_Fy( z*Xs7~>|}C0!N1e0;&W2zg=JFshU%H<&iD(V`n?#HqwrMdj-6v|;d{LkeZ36Ax29r3 z?TXp$l<>V<O>8mS!7(AEui*U= z!J%>mPh=kCx01K=>57f5I*!3Lr6*{>O77ovbRTAYus8Gs{HuL5RE4~qNH51f_oD?{ zx>5E5@@1XgRJ4*G;4LC((klLN-Bl4f;-v5OZjAe4)M7RFU-uLb;S(b$>?uByUyPs! zp5mRkxgqU;icjTN8dA(^-p*@pLsiSCG0j|sg*$IGzob>g^=*h}ig~F|RUEU`uoG3x z(yWxF^Kj-(GjtBG+<2CzhCeplCWpU#{AimTlk%f^o#gPB)E@plbV`Td+S$5Ba9xD{ z?k>2Nb1VnW>!gzb$>J1T>=PQYhR5<-Wt6*yhw__cw0;eb;@8XQ&>G$*6f4g~j7SkA z>cwb7lA=0%$t_g>X{aC&nNRcZte)~tCc|L2ILa1`JTfA6)ox1T4m5K z1vcw=D(*D*T*sq2fvFIPSGCKraC!a?6Zl`gk3`#pcr~;*Go(BLqS)Zw$wNmD|7EA%uN+|A@@1n-7l+|QO)F|%V$Wxo=cBD$9vZs&=NNg zdC_nTc<_=I^!amGpkJfw&+!Ca(u|_l^Ldy)IX<+0JuLnMTGHq1`BE_1CT|+Mfj8&h zHKWxVaOQ$z4j&rprv=cB4LrmfE;SQDidQOpDC~I@TV$u+&-2BtYa!u*m|zgH<)*@T zeLTvKFRf!tg$`x{sR9K5t||k`91KfF06-v zFzy_-b}TQ13IU(3?>F&W{_9Y>_eCC+`r}YmFjlP`^7-JP;%E@L5&nIL^qO~Fl`0je zJh>Y*{|RnXfM`pdifPT-OmBb zwbUAAY&>~n*M{t`LXL&dfLF02d4T*j^Hw1Z0b=soczFnB{W3U=WOMAcu}Yd~%w`_% z<&V>Asmc!{eKUNt%v7`)MWs8PU0RMeUeL}aZ^G@yuYk1Fq+?u&u6 z6z9yAJ+T%$P+!L2hN@8BxzJZ1MfPIE))a`& zw$Y+u-rlR7GdY4y@V-wo2E98}-XO+hFj4q9JM3 zZMNaEQ2rW@0{!P0v)kpe@Qyl>^e^^7`hi*W@BwbOmD{roqhLf!6n1l1FodL)UicB$ z#vzwb(5gHZl4ltn3#Hq5kC6944kbV^BR+A0KZ>;ZWZ%vc{f3|=IgBn6w4t2s+&`j^ zGdU4|e;8(1dPdDMoEqklzMVIm2Jfgq^&$o4!>O&{vrF-&A}qHh_C~Gcl29@}DcM*O zvIF(Tc9XFzbZD&x>1~u>ihr|`rCw;bgGuVU>sWodAZ7ye=Y&l9@He)PKH0&8JQOu% z(A6FMFdVdVcES>vcZl~O-<>>*U!O${r;Cwv03Yl<@_C&{^~|^DB};Vj$7i*c=#(7I z4ytiQklVE*Kw~^+nM(zgXrLSLq)Jw($}_1Rt+8My`QuJT!Rx$)FPJ6w#X&UY4Suq1 zXOLx?TySzU73@+zS+WR|L#COQ!@mUZToF_zf|R!&Dcd6RXu>X@2;(mcRoVqyPOuja zMUy5yX9hJ-)uLEqI=c&dbuUWZjg#3F%Gu3Z@vGBm-EQ8*?_sp-K60bz#BN@UJzAjz zVqDPOO^?0FJ8LDMjSHjq-{g77HHGOWixI>mk48B75;l>}I>5G>6t{;rwPRnP>GBSM zg|)pA0GHUk9sw%|`2OD6Q-4n4J#P;TLXXm0d-$M$-ys4Ez&@XGb``B=h85}<#dC~3{z2n zQ}gyOUI&LiMXvAm@%A#vXG6musLOu-d3}UhXo$`lGi%CpV`g6PU|<@gH~+^(!e%u>O(Y*oBou^AVu?u~PG=AB7eapmYOX2tigOBGMA|dc zsNQ?Lv(<;gziXMWRcm(@`M(VAD7dJ-hG>=TU9yiO-p)Z)YqpWXBu+C4)av6Y%uHl8GTc|{;qC48OF{F{Y^zSx$DEd zD6IbZw_<;8aD7Ro( zodc*_FsyRu=)wm)H%Vp5q)kvITqD{C(`aQS#|zo%MN;nA*vjW7qXQPGdvt;iF-Ezx zk^UQfagvAm90%%hl$T+Yk5a^kJoGk37d&rcWUEt(m(duE71H)I_e0+2Ur_txLq07* zp;iZ|8DmsmccB$qlwOnzw5&i&F(1&Hc9Ku5L2IFbR+bxDAXGV&fM{A(&S&|LgHzQ7 zjM{{x^qXpwZH=k^DIOQChN-Zzn$N*;|CrDDmYgPPK2!MtZnhL6JO{=ki58yXFN7Zj zOS$Ir3g>(-L)z?nUGui;Bh1?>JK(?S@c9_$EK)^P0a}8+;gKQ(fSB}0Xuv0!()H*T zJaGd%XgJR?>Ep1XY>JmEgGqNZVl?(Egd?AAW*TLE%6DlW$R_g3d9cIw8TOBN1G}51 ze9qsC2V=+SlPBPC5$2yLFnVWr4RwM1U0N8YS4JDBi6)HYVQnuc6*cH03O`)i3DOS%{2}k+PVsoaN`W9^*lm>1#ynx^K9z$8uz}9Z&5Z z5D!!SH$2@u0tpuS^c&Pg;JpeS`VEjdZ*@a{qk%m>xnFgfBV$LLU!nqkiXqVzZ%(+}Ilk4*&+q+rg>&P6Nrz`i7eA|SXI*+Y z^j2aTjAxG3Jt_1g@5k!FeJSG-5A;>6AdyrJmNlp9zj<(ja=QuFZh+D@m=xtmq>_G# zH*xFQqGoq?t+W|^e2KRh_ZKQa>{^Y8p+<0IKZLvVm1)RR=>W_Ss7)STR=LuEt~~y0 z0~3LCG5OJzjD#o89sku z6w@JPOxbpQ1+BWmnK5)eS#2*&&F1U= zq_mc=1w96rZaF%Cjv?=Be5OYty7~1Odh{CX4&`Kn1tAc39k3G4$$$o)4m4a%ynBu3 z%OZH-txhS*xYBd4^T|fjm#*^%VN-#Zri`+Xs@T62*VPZ3^eb@EbS)Ss@FupT@KE%@ zm39_sYAXsAm@-eu@PLw?6XkVX&v z&Krdrj9D@s0mQ)BB}>u1OP9%}y}$EF^Ds04>HE7p{Gqxx__BLi0Q#FJO$GA+y#P;- zE`xp`=VTt9!?_BZHwI$hl&JL7DSuYC2EsQlM4M8fIF*>Pz2r118EKIKUDQ9g zKd`Mp7~DU2n0Y-Q_M#JiAT*{L1uI$pgoA6j1xpKjUjl*@z0=`Bn29Y1w)?U2bi^L- z82=})$2gc|?q9qu0cM`J+-wq$XeLCX1;GXnvwzodjKUcLWerV#ka5^xMacR9ys~`&{XgMsQf#q ze8EZObUbx1X}Egzcc6S;3|iDp#7=?!;SAyxtz-kXdC7)sz$Op-N=(al0bNdlK@?Zb z<6Fi$O&pSsE4lLyIXQLc1U$yKKj0mU=GkKv&OKuX7+ zM1BMjDaGOAbe4&D^ZTF~ly)wLg4rf;k?x7!ZFg4C z1azA$mqRRTUd|;e{jci5Ip}9DrVn22X)_OYAy*j-%E)OZU!5B6MvG-!(IUvT<%bT0 z{Y(mcsSSS@i;cDm;Xyrwu-bddbrjoU`8H$)-iBluGAMA_7HKnJIF6}93x$ZnzNk=$ zq=esqCc;MV#_Y{^QYn9Yp`ud$Qrtg$(bIHFQb`UGRov?tuF1wJToVbN&X%aZCc*)4 zx+X$+R6ISQiL$28{13S2{xqon%Luy9EIMKDztb!-y~@!{E-3gY-7LT2($hO|L{} z4p5P&Nb;?)I!kxbD*@#-y67pg{A#7bRW0-;MAT%9u=2l$(4!Xd8NV@tzMd=sDLY;Z zrlnqDGW3lrULrB{ezd5peYgsM;Lpm+SBCz~9xmhc22vkyG0)?9B=i|hKh1{U-5GBY zqm|4Z$H=pem@I86aMgvr&rFL3K0piWh*2T;f)nJVMh{?!!5fUu?2&Y}j>zQcz~y%y z>d2}q%&bo>?EA{Pa%d}pWM5$wfuj;A&>iU<%-t>1fUQ;?aPNkW#&N(8=*L>xdLZ5KDj|6#WUbu$~AqZvmiTj%Vv( z+9Z`M&DQkFxzHY>jdC`W8`>44iu=J9*^a-AOlUIXMwy^UUlHJ=1ceFDMFMaxW%%Nb zS~(5(70=k!a+Zx|0nXhQG5{cDhrS>`KM^nIDMDdtYDsWH!(E9?lir3V*5v1yiTuQ> zh>wTdNs!|zF1Ua3K@e;PWi$|pasuSjyapo4kN}%JlmxJ1hwouX0DyohkT126&l`xL z4L5{3xzD0#zunn{t4wcyF;p%_FZqiRp?@P&GLRG@U*jnTxeWD0+c$>Do65n|+$y{c z3_4gv3t4TJRdkbo9ab^K?2V#($uB^x!-U)tAU1S-2PXx+jw$(F;=agKKz=_q(0IP= zQ^@bd_-%8&A_m#}HPRN+qCnB!yb(62zK*g$kp%5aWL$+(qdBOOgDMy zIoR!NaHanhZj`rq>zgpg)*)T;vYObXUivtEtKU!@#?qKz5$2JI*1jH$E8XHM`Y=00 zypKN}Ld7Kfc{x-hd2Px-h<0FiFP#k)(y7Z5CK8gf1^|=l+iS;ll#oPRg3^|El^njq z*()7^#4>40n21E+;pJgs4FH9Qi&prP6)s-FpKIYFd9bU$g>_Vahr?bo0`mr}m%p^a zTd|(xc#bop&V`A(rzD>6+li~v)-58+oOhQ8untRtW2T>*nWmaUN<3Hc-_}$8n z7XC}wYRL>3rg^gbO)73EUNk?9p^Bpckr39fsYQxL_0i_a0reqs}mVty4VLugYIvDNEg{ES1{GGbxEpI)0UEHpa?i$I5d(i{*) zTVg~H1U3N*zkB(k@Y^6(8C{JP4)eFDw}!UGi9_abNX?-+O#%ERqz+d%-eo*ZX$gP(v7=x4>l7A z%%w=0MXAjJ?iJe99CG&sr1Yb=nu}=>OYqZmC(i_B0NWoWRu&>Xfy!EPZ9o3;V_clbD3X+sM! z&wLz+JxBp9#dz5273`HQMQiBq_&r`*D$!QnQC(VzS@p_0n{XVgl+%FXd#S9IxDRRB zGA+P3*m7jF76Um{bX+i^HlhQ571pEA+lZ`&*bvv(PvCyxI534my!3X($Ea(H*oe|L zA3C2RMq|n7(pEf;gghVmsx3rIJe^1ty|nV(=X@Mt?L;!`xvaPGWRqH7@nX+Xd`sUl z^^G+%58v_@wfb(AH*f*IWn2x_wh7^fu`XA}q~DxP&$buu#(j@$L!b24*J5jZABtAg zqy-n}VrA)qw6AD+2N4F3gjYI>jz2E}xIM!8sBVFOQbQhoC z_-J7d5yqeGMUQq74IO)Wh`J(RVlp%~yeHNa3ok^{z{U8DcQbm6Cb?Mig);mNX=7}B zY*>v=*euO}f+*j&HopDzV(c@pmoUE5t@^9@mKF+77fMlpehTH~b5cfmtF9v-+fXAP zKIF%d4~@%NzJdM@zHKIYw2ug7pVPB_L<~DYWqm}eh*IP*lyoh#_bB&fYq57vMXA2x z2MD+ueMP*r#FXc&|Im#UQ#}Ysv4-#c?URXO=#F!fbNf2?cxCa!n4H1JQZnnh8?=ck& z!6>;u`D%k1VlU}KM6|sG3W!UFAHzmQ%5Vd5++u;OpXvB?h_EoJcXMonUE=3b7y4 z!W=t?i$grx4(RsMK&1{o=>fIT?JPVNd6q>%##y*gFi2^kFCxyoy1hSI^ayPWIw|cL zt9_NznWh!e&Slcf(PF0eu_Ob}Bbo9<4$lvb5nF5>Y{nSaf^|D2OMPhJl6S#57`tFy zqvK<+e4QY6kLYgR-$h3feD8eWq6l0}b2Xr;*}(=HsOp2~7@!x=jrx<|BN_d2fL zBkFTcMYf7y^2`yN8vFuPLBdy(%R6u2PZ_i?M;v8iXwf+FwAV5;pMp_pgU8OqM_yHu zAk#IRPG4@~5#qoaHVNFP;oXJQGq_dO7e)D-$UWdoB#CqvXGgYK`??g&A z6o|3u#`Wn>riv`SG=nx4XlUuLZczUW?}$ ztEP*>pd%feg7~lMUB6)tTEJzNO(B$4&m+RCHNs3c!%d*AbYg}G^ZFQVFHU0xkI>Z_ z5QbQ`W{PlLH`?*dOi{vet1OivdXaQ8vq!@Q-=U}n!``AruzvBdu|ra@S(wvD*$m^i z2X3vfQx{s9ZZfoM&kF;uF?~cQrFmQJ&<6{V|4F>D_Ne-9ZOG7wD$O1Wm{f0$2*F{I zJdFpCOlmPlwBpy&=-xS^VbjaNyAscGW|&O7FXHzFyu4}b>u^mF?F>6$SQO8t(e^o_ z86^4{RM$#iEFk~;M3SxC1RI9SrXVliwKL@pY>%2ah%Wy<3U;&mu*Hn?O@M{t@9N!{ zdCIUNF!!a&K*QgqrEOswHn#RtNuP`T>GgQ}Y_4c2{_Mz}&h-t5?h4edb(F%JpNa!; z3yZt4#dO0V4^mCgKTM_w%A6-wnL|+U7=1cVEa>Blr|YFvJ}*N5Hrlunycktuv^!la zDuI#RWOx9}{3U`}jsC#D(SupbaIh0gIXLT~UAVHwx8hYJsmWcQFD6Iq!H~GrBe>@;wH%H<3DZGUwr2X>RgJN0o!Bhhr zRNEB!Mq=JLq5pzI#Rqnn3gW;CcN}$KFLoWOFKnIlXE&s89~NElghbpUBF6F%=o<^RH(r{V{s`9dDmbKzlH?wM zc4E7!AO+Cew}a_G!2*=Aqz){Hgf`=M_ASp8>AZ}Sw*{gdt8#2vAiA))uVRq}5Xk`H zF7pfVc=7tS28gWy@eBn#Dz^H()(-mjJJ*D z`q@1fDs{QE`ZtrGhjPVm^4})>TKEHQDNgAz7s@SV4GoLpX1qEJz}2~;=t7X1U(gBR zOMs2kLyi%7Soozj>2fXzDE^Ju644MUUzCkrS|lcGCHB%ts=rvYNx9HOE?Ctydk($^k*$eUn&w2%&%-IW`9U3J#~d8 zQsc*kykV^;_s}% zV@e{}nip4d=p0@xr+`UsNgbaMjRM_r>N#^7-7wL~`{B};|AfdwfcDBKfCU^Rm%-ix z2aIK+6;vb~`@Bc`GR%!~`ed0Xg%iu#+vYk83FOQi^e>C~uY6VxEh<`>5Gd*l_Ja z-guhw6jq!9`u-{LM#D)+ai8SBt3kCE=OEoT#n^p)w;H>zh`#`4D|A-Z?rWv9qQ;r(+t^Xpe=oce_zt74PqEu>-c_yc!T+DZp{jY zB6$e@4W?H%iv3XVb6!AYJX`gG_!Xz>dLh0-Y0)O&T~R0s`IT0*coW3aNXMy7LNKpq zP`5kOz;2GSF9N2l^0Ig!V1ElI?K*RP176LN5468Y5wD2-^*%@G$C3RQ{w-}mU%x1V z9JgLUGv-|6G#DkPh zm)iAU{FY`G^^IAwhURP&K_QpWU1Q;f#nb6a@&nRfkliL`0w3S)A~nW}6KcTiTpFd9 zL5K8p!3CTl&?eMc2Lx&Mb`i@TNHA8q1KY(s-$j6xgyJHIRQ^%bmTEk+HPZM^E4H2J(v4T;XOa%e8mIrBYTt^YF zi}-*Qz|Z!FVTM|UEy0IaEgq*aufwziV!tj5;+_PM`gZ|h!^uc5)RxQC=cv`4M!ccc z+zoGt!CoU#3s~`0?-FaUO1N#$R?~pp za}1;x_Q>v~90( zY=MEV=~g^l*(>@7JcWh5)M*C10Gd96HkZ?YeWE#}Zr(nj*4NGZL?3)!bAE>Hhe@}5 zZ59jn8HFC)LKVpq7DGkauwPj5vfa-8;t8JCjK2N~p+BT9O4%9<_c*!{56Q>E3sUUU zOrWe`jT-)zIAbmaWnhgu2+@^~pJ{aSpcuygNN@~11dBqwCP@1f6#4}JK5nXL;5bq$ z9%SZ&7+DzE-&Q!!ep}?ntqa1A_j$!k!=fF%bd?9tgNH?U z{YQcU3L*zLy49}zrNf|Z3B?`}U&Uh6-coM?dddB9B_5E$fpLkYvWb3iywu`56k+nh zR@|84-w_d5b$h-8bIN*J{tm`zIh}e3JK>YS9cNVUU^?R9>!?V98eP_iHXjw&VZwR- zU1)I=sPbLW-#Y^(tDrIc982wvi40JG!7;Jbs1;1<$1!(sI?>26>A3iUM|2B-`~P#M zf>7u=BjFy>!?`pTR#p$jyKNtc-l4xX0&{(1FjuTFrTztDOA3_TD~;&w55!UPB{T-3 z<_R$nZB?8Qlf8~uS;4mkJA4RjvBaL&*zv@NVm=>Hic@9hOwV^slW|ROIt9Xf9u0=N zV6<}~ilt@(!beCmIT>4~y@fOdBfq&6MkSpq8Y^KmfQFnBqnJ)RPeJuWXf1p-K8*1y zl$Hxjq_?Hya?Z;@G$<$1H2lRr_akvHt~&4fNX%m)ef*j7g7rhY@}T+9x%AX+Xw%1$ z@rhh}%*dC9ez*>NDuUP%$HY%X1#UgPop*LV24G`)#_4*R7Xh{^k*bG(S& z-3^g$C`$tCl;ho~7i|qKlMl?x{;r zS6+C)(t8>_8bn{57TL864Mw3@6e^+tU%>8!ySrbA(X|UDppYDkMRfHGk>*tfYRyIx zX7iV#yH|U3T;`6V<@m%!>{DMtAsdB4-YDcnr@j)?q%++a*t6Rr^?WpqJOfP-7RfW> zK<&PCr-f%lB>$liZ9FSl%9(W*Yl<9%`zhjU7|Dz96M;g$#+GZfgzZmtzk$v)pK`u| z&NK_T4IK-=!3u4b&BMrvyAU24e!dZ9NLKkMw_`%3urWAqJa$e*^FdL@``b{b&WTL( zGVoOj+0P4qyojn%(zjx%M>f*xM>zwFd?$vR6Or&WvG37-{&%o>S@G?z;Jti7s_WL` zdPsjumYc?ZiK3a`i$L>ba03L#_t0QV@e^eHL8N%>#LsNIVc;nL0ebUe5{2%3Zj`cf zI2N5pj}f=*`wP&cUbWrUqnKQ;CL?jY6$z+!E8y89Jmrj4)w0$^hG*o~Re%@^mc9`8^})5%}9{;WeA26)9hAq zeyI~ZX-}+!Jn1)c4U-=T!9MQL>WQ4 z{DQC2FCxXV5;ScOh?Ers?}0|YiYGwZ*M1dmqvW@TT7Cgcg9LaA2XBIL>CD=&)c z&Fer?Lkj+iHC!r2FLdsly8Ok?sYN$D+Z;u@sz0Hn(Is1g+TqS|qW=Tx*qjobE?g6_ zW-p+(hXSsP_q<>bO3)2dtF>^4_L~qq7rk$;&j@D)5iZo)I;mBWpqGJKbCHLLv~uXm zP1()$arkcF_I(e&<+;k8ly0Uhw-b;j&rt7{HwJmI=s$i#jAZXnd7xv- z@K0R(r)bw)A^r%|NqOL740&_?0es&LF(?9miugg905!Ru^D+8*rkiQ<+x0lThv{Z} zd>d-LwfL#ujV*o~q*~szocyY*dIu1uFepgwWb*G2AJ+YZ!C0EEv8vL<_QHpl}s< zhzfI=b!{K53tPBW*J3mPu~IhI#!v{;UP~}q%6GNY0yUh&oNua?*Oaa@EfG@Q&b3xv zdw|F!l)l$7hHG_Mf-wzhl$B#pmumpk$}6L_0)Xn$J3?#a-5-GdM(JB2#5HZbvr9FA z9-(a-q5?%qf71$0^Yw3KG3IwNMr4yyi45*ui1y#O&^N{_ebL<23xQ{~}q zopeHNfga@Bv*DkK9vp*uj}RqR`TGPMFY}~2{7i~Lw$fyE?v7dLuLU)Ez)AB`_y-owwXQNHn5JLn7D}*ca=oS$tG3X< z6M?sI&C04h;;B%pXhmrOTB?zk-dgviMFCn+C;?S@1~3^;sY<4xJ#|YfkX{eantH1) z;~e)7EBz3lMHzj@f-fH&_A&a5HD9Q>bAej4CTSw&STM~F)HKU4*prwb^wwEKYbpuU z%#wRvblVkNx6;u-Ev*NngGzSFs7lmx#!E?+kwCFu9lQusv~MutHI9QTc+giSt4dpD zMEe|vBd-S3H%NN`YT!FT+FXw`5b!4}bqdz*jcA15?&{pGuMBmrA<|Dc%7QiQGtPh* zE6_4_G9lVD+aY|qsz{HD{?00Ukv>zwHOxljCY5Z3*`}g4&Z2+-nQXY3&(ASrFOuOpMmrZ@bC}SaA$%nbfX_ z(%Bddr%AqsDC`ic4fk4rGRJCF-V>{J2%Lb73p{Ye7H8$U_}=d;Zy+}H+zTd4{Wit8 z@i{t9`@DWt=X4o3)3|~7t*>$8b97TJIHClYN@?S|OMSo78dr>TD0NLW2Y+)`iAzuD?!WRb^Q$A(C}v3Jj+F2 z+ySi*v05TL5M;fXc0BjHdXh;HbvRW8J2oR1a--ZdyS|a0-cGsJRo8=C(2}qlTcX*i z<1lE*3q|sX^1@*mKB*%#U?{4a+8IZa&gY)h(PWJCSr{8)-hOM+Ka+4kBg-aS7RBO6 z)<7+w4}!YO$^1Steb6a|P-#bfkp{mQiOd#%U2Z8X zQ64c>GO*pxbftgLL8fD*Q9h>DA+N-#CP-%0lZ>h~^xo;ru9?&l_9rKaR!u=yvk!pjF z>Q;#f4$Lau>A<|sl|EdJH54sltn*#zFjK4YPBZnBNUsB}Uq*^Jjo%jhmV9UWvwGKc zfmaw>YhC?tQ6__1hBJ63}XzfGxgHmb*&NS)r8j+;Jn@F2TIc>BLT3_}xFs*2> z2>p3oXO9h0hn^X_<|(90hPWR!vnVG843UkWUbHtwYweVaHHG1dmZFtM)&emLAga7D z(b18&@Oxc?XRxQi<*D>@b1l?@86%mt3pGpCqOBH`sm?T=!etd*Ua2?^esD}k)z-40 zMUC`ctj>GA7kaJlwea|Q`TKpD>)T#y&G@-L9Nju-^%%oj#aV|{@Wo#GS zO4ptOz3lC2bq30o(~%4~3(DWokjAq5q_*2XztjzUDz)EH3$$)HLX#=dj-Bss;dXy8Kr-o9+f@b~i0ZlLJ6w2WWwvodl0^Cpg-G8Tr2r@?8lY z++FkI0cMKtu1N>V%YC#s+Sy$T#F1`cC;d}8)kCueG7rVjmk=07zND@g+TMv?=&3d2 zV@eHPPUxY<7?ocC_ewdnD(&f^`C}hc*ojW|&;kSQ)()eMJ+;W1W|Dep0ZC^){_)1M zVvy?|iY4!$5PC7~-w~?pSksF`J+&rUoR_{Fu}w8;S1;@jT1@QaqDAb#Lvf>pd{v$* zdbwy}R+aABv5cCIsXEhpYyJlP&QpuNn)R-A7S+@_*;{M+e@lmh3h-S=(~S-an`M18 zKh3L-?nP{%7EPsnw0MI%(s1b!PhlnY)dKGd$c+?A#O3z|DP$FW#J_i|tfpJ4PIy0% zLe**iZy1+P%Q4lATyDSrD_SVX4gL)>MD=Z+Q0TPnk10}FmwNZt1{-XVmZV^u#I4d= z;Y@~3*to~@YH_=QCSN_j{%zh=n~6+pGT^> ziT|y+d`ZFjS%z#-7+)R$3ZAG()dRHo{+TCD1(6!=t>HGUT1Z;dcRVprOJF#(s$k?8 zq$S{2I;Zo4G@Crg(x}d0EtI#YPcehFXAvf;Y_N77w!NYuT3Eo&Yo>x5aOjj;QUwC@nuzOET*V%fhF|Dw`KEk-b)ih@S|*`XT9qDa)fZuBsnRPny}sEn zX{v9Vrb{m=^&5_=##_{3#b2O|-0Q3KJf_>wx(wD-q9I+ah^0b5L9YzO^ubBWP%Usc zZgiSeBoC-2`EkrwZ`wG$NJ-QTvka_vHI}V-4l(WW<3a-Tbhi9C?w5Wu-4ywmyxXVJ zGXiz@n2;)MQ6Qxa(?Yb>AK-;I^*+(;VVZCKSs$1R;0L|y4Ja0K-}a!@!?b|l^GD=E zWr4*wAEu>mz^Z z&=+pLi-D0AVko;KwQ~NonfNHJ4IVyA9i@HkUGmwvBwd;m`;mRL7K7k~Jx6Qz=E7sX z4cwKj#a%Ic_XA1 z&wQA4shKyUtM|a(mxF9y$L_J3Co?z1Z+LEJYlFB&qwOn27`4dJc6fY>>PtCoUm>E% zGEO^S-hhPOvY?gDj?>c2kK;R;?BlgzcsBN-@mhSqyGY4?+jOcY6w&!`HI=)ev9xzQ zEEZfA^j|kYo1nEuDXJI?+d;tu9LYt}-U-@Q`~t6eZ_`TeOw?9;?BTHQG9z{Vz1qj- zN00zJ!X#}!>rGyhwf6k7iMmbJhMJoo=Tpa~$=V=hF2~^(ZbeMd9x<2T=UB%>Q#G7i zO?NytO$*}u2NP|ap@kYoskjUpdq4zu<}qg7{FjL$&mi*qjTzbowuRQt)S}HVm~dCd z@zzX84y_S>EA!2P+1dwj$1o2qasMA<-yT*~(fz;oK4;I~APC6)eh#1rh={0&Hx6id zDesz@l^~V3O3g0%Y6)IaGmD+*O05LVlv5^_wzD!pyO+J7-K=~Ct<=oI@WX3d&4Yx0D3S|Kz|oZ+95u73`OJLG}k<}7=nTx@o6fC9e2u)+{3g+(LKmnS`|p?3o#IwId-9auN~e2LziN3e@We!q9=CKavs`2l}qigTs6-u zwI|!`)P16s_tp>JHY z3QRP2ud+{8;Dp?{23RTPy=&}u$O)M4;q^}Fc|zPpSRBMQ7G>XseHt$h`$e+<3J+i+ zpXI;NagO?}!^yy<=k2%JE}w^Z!;e|_tie1pYpwl#?i-B*wK9K9eb?Ei+cvezHk?A% z+h1*+4RzCj_4a7n=vLVRDf9*Vuh(aT>-HCs&DJX0a4LK8MriwK=u7r!` zzGNS#-2EPPe%TJgx|i(375Nr(Xa%Rl-+!6UZM3hGWj{0hWspc|`jduL*yGHduV9qQ zTg}5;?8JTf&#J_5l~^)qjkl(Zy z$;#EY%=>oQ^A+XX@22}Ldk0x|n|1I|FUudAgLm_CZZuCSSe`1}II+xWBfk?y|{Z{4yA z5jVz;p?e*tEjBn{${+DsT;|+Q!G%iGDRa$d_DI_#UX5i!oNV_PWvG=W(}Y`ft5yup zs1s7zANXBgYmBJ%BI3Dnd@KCeaR%gbAYa>KfDLD`PtBk&?E@8gnK{0e^FQcUXMf7Z zEq#21VkHIbs}k+ok5%qxoZtM?QWGqRueC3c^1ikQ`wm8hd?;M_#k~7#``>^}^f@bHM$~zCYUkQ6R(S90lAprnBC@O3|i> z*~P+nAaKS61Kfb~cMVe6wqq@zOIzc-3-LGh$qzt9jOF7JrM;~<&7Q~Y_t=z+$IO?0 zfmcHLb+hJ#J<&GBQ{n&W9jkuy&KF0evWof}>yH+_Q;B%dJHFTV&X>TM_s%^0TelFW zoQ7=q7Og&Q&&FbK;I#dsJk9*@H%yhv`FdLSySCA4SIzpkpF+6a*7kD|=hDR8swJ)REW|e#P;`%5UTKp=&7{ng_fgPQ z`$gN2KcZ(=E8XQs&AetiQ!-(w)y_!#-lDM!wMwpFBOYNfZRX-(bA0n@^DH2jsFdI9_oL+ASleP~yE!`59^)lEU& zTY!cNpqaEwRdZpIbY4|wPuln0|AqZ6>I?oKuqCAN&4Q@U5g=?9K1baD!sXm|^cho! z*_U$i)b6o6sE?)&3aUmhA1y5i3t7SEjrVA^re=h0Lv#+J3q1rlh6!#f9oE#zFr~}W z)nFJ&jn&n=@2HYacC*@CA7B(h_3{YXA!xC)yA|&W6o-j3yzw{CXsS1UL=P)}d7z+O z5&8S5Zw7S+A`T$IapN;;D%HW(E~q1-Cvp6MHt(P$I`5-q1%)8G0zQQI;i!_2@nFjF zRS)J@NsS)hcrPmtNFM#xgXG)Uo(>BVl^*uSdy%QO;t$c#HtGUAbaJ4L8jsO*ri~gN zTMlYZ2@79QujOr%RB2m5Sm=rb;M9NxVGTL`)Npwf<@l+OLE6~ir{0%ZC6)HFsw>7@ zW8jZFz17`7liMiIUmYj+rnUZRwA{_y<*!auVqXBIPHt3<(7D|%N&e9*^8b4L(D^i1 zQw8Flqs&0m_bd$#RHGx8Am=wsGL{8|t&p6WA;X@7_uxW$JW!nud(cyX>M?92JZPZ1 zu;1(@_L$-u=y3K0YheXwYovT*v08b5rEU=4Cl$igB?Ln~NfK*T{AFB!(cwskuO#LE z219WwcN@;}WG;f(RWW=b^R2`{yvmHJp(a|$Z4o(cvswX23 z2-{l~m0OB-vz<98(ZD6PblZ+EEwq%c4$|G>YBuG?p<=vr%}c*>@LHob(SbM+Vw9Eg zXseXjv@#xLCeo33Ad=_-DBo=6IY4|r%+IA#L5Q)t4n5Z8JIz7$O`4rz64e13I0Vi} zLe0sE>ZeK&_@gwxI>c}ZeMG4HBg*TbCdxbE0731fIAhFJ9n|(dJ_2BIsPUv(lc5e! zl##Kf)>VDTrtF9_=XFQhoZR2WkV<6Zz4tAFy**XTRKwNyYN_l?+A~w_tp?!pee=pp_1zfzcMGJDRmD{y zRJ&4DldyW&v5G9l{aS|qkEBS$R&edFK#2%5Z4mPu*<<_ zjG^{dlCXckZBE=beBmvrOhzh4#Iwl(U_%B(cH zWsTYen*|5Ia>j+82=`7}zDCVfet(Phtx^9{8akOKshNs=$UL=HO|!}KDD*`&SGly& zyzNDGk}TV3_e)q;0^zw{U4d7#wrxqp4K5Nrt z-irmvwEh)!s1un>^QB~Dj&wKRu4(%nlDK6o;$4Y`lYF+Q6SRllkjfHwNHqMQ%}Gyh zQFD~>HRi`#)R+D2jn0Bpnz2)T9$(d|W}98A9XE~nXYgBUdli=SIq$0dyS=;(rk6M9 zI3#qTRNSlOx++(tEVD_w#3?A!sOnvntEG!avKW8tn`@;~sQTdk>K;ZdPsgXU|(m9)^fcsewW zi!@B~&dPG}I)}Vr9Y3oCk9@5514C7PtcEJL`q9T9s|m_oHFWx8H5N|ieLhiRlrld` z{zTl;9Q%nn!d4n?z{{@|LDiqAC!>#p(ww#4X>Q)hJMG=qtSRtFB{W)NaP5W9_uU$x zjNP5l9eO1s-b+H)3_T&fYdSt{#>Sl6ZIZmaP<5kDCIj@l1t z<4#gPhw)(%u)YMW`L^p0d|&i@-$ipjSMi>Kvp9oZ{ahW7E34`k>ICi|;0v{9l+z!x zisdaM-*ejg37>lp_|cnRs3{N$k9?sHOQbuFD2%~Asy7Wi zppL?9oY#dm9Z*N&L9g=%u)Z#)xnHWQ?4z@M$t%^CA%e(h(k;=|t5r_6@ zM6DWuSM8?Os#$6WFxl5sQLFa%zYU+|inJ@(RXi$+<DTN(Fl7TmIfjr~s0WERL(Z zr^EtVr}luAQFVal5FpHbeltB;r#{Z_CtHy+U>&5OgK7qU1MeVK`X|UZs1B5g4j)u| zryT}Xd#82l$*p_%1?Lh7nyZT>So}m70wEj5S|Djt!*4U#Y{>#-OUB)4KQET4YsKfvN&U zRdPc-br7ZC#MeJHyazxh=2y&95I9MD20j?YfC=NL-n zqa;VBqQ6{MhMImZqO0GlkzCc*mg0}75xBG<1sLT>uBN^}K-PVf?)m}zH-WbOpl16= zp+UtyXb@-tX2)B0$xUPNb|#!)=s{hKH3@np+TWFS;8Q7>qb7@b|A@J5 zG>!jJy}fgT%}AL$<#vptz>!8Civ|KkB59+4ikm%$Vsd6iqqTnv~C@-2y zSAN9F#f_MvNG|0aQRD6BU&fTwSf+HOr;n(CFt~m32$rZ?^Dq)+rT%65`#;!_%%k+9 z5O5!&kw>u^!0o)FYBv0KeSK8@OTM2zt;a@3qbv0gjxN4P{TkF%J7!Sa>T6JO2@^67 zEo)FGV5*wgn@%*SV=;&HJ*M_nq?gIJ*`7^15b(c%&7Ak(=y_^7hCwxkoX6FP5$FSm z+x+Y()Lps}4to_ibKL~I3l`a<=#Ar$MYhn9TMvK z+6o#@(>Szy|9_!9&Y_K?@{ zK=V@y`c;kg+XwQ^0wKUXrPT9RHH9zeMZcweH1F5qDw_hfI_(Mv2=Mc#0A zjtEaQN1jvL+D5^W66T^Pz-8+7xS@Lr;;sN(ZpCwqS0nzf{}TT!;zizij+h1GDcW%n zveB4~{2NQ5M05X!B4s80%tMDL{U6AH(`eQ|YVw%L0C^I^ zg;c%w5+sOgSArkSxU3Ffja-8In08-QyDn&y zV9eTUJ!Eg><#JRR>ihySQ_xyV*8d*iV8~G%R(#P-IVan=noc$~?>d4!hmhuV)2SD! z!c8aN^LSx#vbTMai-wYM`UOi_bKwO@G(zygNPR)KBG0~HK6C{d#lVOd!(xcOLjCle zx%#Rar9?K)&R30seGnlb5my6(q)6Q{U>jF|!ji)U-fKuI+kq4p{2jCdUOy5snI*P6 z+s|;rZLoVwzslthqf;=S_m}(Ns6%BSavvX6tF(h=ux?dBO%1D!`AYr95)<i-wW@5uL?BbQR--A3uh;Lpb9=gUfD_2hn&RHa+$O{FnnG{Yb?6EJi`D z!F+&b`LW(mn{V@Dx#+wzeyqPdlQR8TJk(Dk{aI(UxIq?9z^L4xh2RK#R$G=v_58bn zcC^KNc6wVDqRs>fXOOcki{dw06<#6}dH`h31*!|ADBc z1HFhpJH~hw)d#Y)0Z=c&3>`r@$Htq-t|kY}(fNO_te>{~xc-J5g`p5&$Uz&{>0lfk zRwk!_3*X{m$`)JO;Xw>GR4HrIX?74B84grS^}IksFUjo*ZB;ihVj3*B$KBx5SP#HHjp#Jni-<@wN- z7&bU4Xth+vIr|$_$|oaSl8Lh{C3;Kc;@h{!b-y@{ZdT&v?89xDTS%@YCMY^ zItM$Ja#iZV4e2pY)$$2!+9b*NHJMd@%7+CM4nx2pr?65~VhfT|k|M6+o=US-o?0Xu zr_z*4Nwuc$dnw4l;yTq1aXJvsDw}&tO@pDNIQ5q_SHP;c0Tz&$;9!yRt@MZkELP>5 z#<@K;OZWB%2m!gI~bh*mr#k5x&2BxnS8N-<7CQCU~w^psm7^^a8{Tod8lBw zVC}~(MVghsA{18ytw>?B zB(V&67VS!6{XoKkwsbLx-8=rmb5hwdCGQO%sqtXE>~k6_!!c(OXN@#I^h1R2!-)ab zX!{ju-s^)xk>~U=^5;&y^cfm7Brmz;sr5Out38{5l_IDE*iXUP2NvpAL$`W-?NyVCkdAO85NCg%m?s6(S2UoRc(^w*$5{GuexW$QdCl;-@0Jx_UdtRPI zR}jhZO2_CONqOnlWl3i*;N)HI3@8!QzB974rJW7PAY6|B zWS}|~|2`0j;}8eG@10q6&P&h0Y6eLc(8YX{QH*6x3hBm$+0uL{Ln`=*GxBiP1Aie8 zrp{8yu%XGcxy#*JQhOUh*_`{z>22xTAsy;{$Gw4@75Q+_I?H@dJ?_}Mtp z1)VsP&Ub-S@;QZOvdOnye3s+o+Ra6$jwQN#E7WH~*#iGWl~A9Z+9YOSXy1Y?kqTZA zPjL^3iFj89UJ$V;l{(9V46Y=M1@fUl{@AnhL?#RGc0Gtvy(aK~gODr`I>mNn{qK;Jj*8UMx+?BPF;UKFki@Ehaw68!3`MhMI%dp{lD{}JD_X(E`J%gh-SlZbeY4yQU z_pcGZ)w5|T36{qApq+eRe1mv8*A+vhQBa`x;Grk34V7<^dh$yg@%TwQzr(jVC<`P5 z4pX{eG&Iul-PrB0MLgS$MQfMHoHx!YBIe}W(9v3s!2en^GgvBF`i=4XVX96#X$m()s)U5$+>pD(pFoc7{TE?V0i z2>d{wc4vv92SFiW_=Jh{*Oh$633bDXZx0ro{xB+XUB2Xc$1l(lwU}3m8d1IdHYk6J-;(XUI zh?BSk^Hs3aBpPvJ8Aosn+)|WrV2y4@-iC7=ymyZt#M=+ewD+Z&ndppm&q^>d zFtS-ce&EI**WhbH7_5ztWV3+~&D!_GSlmwId$Rbu+yGb&v9*4jv^)NGTnvXNnsS`e zo=5yLMOuM7>~*JX(&t>kBg;Nd@#&*}g0#EbN9vBruryyv6f$N5(z&X5E7FAkiSSDL zx+hz|K*A&>Xao06*U_dt)ZoI`@kJ28Kx=a;^g1`@DTaCP>}z>mZ=I(~?6}~k3UEeW z%d@$4p7MYwiwWuhsOVaz7m&$wToKcYMPyZjs9wA!PbW+V;;n_@`<~*R5aQm2y!Si} z(*zB>>G{3b6dV`)+>2$YnJ55n(7o9$kslyWy_PRt+O606<6qv!w{1yO(wikJS60$1 z_z&Y#OOaGpxhVG1Rv7n-F^F@8qrDlt0q{ZSf|Kn;$#|CJJ}kWbKtR2)0!jt!S-x6{ zb4Qzub4LZ|j+Rw39~P6SZy#2)zycL+oP?vc|H{31faX|{B(4ER1MdKEofF{oR+H%e zEL(7`te>ZM1t$aE?x)3LSxU$*a`t7xssxzZ;g+9$)~Qy~yI9izo-J{L$u5Cgb-Oe& zeziR76I~kP!qfnXt~RAI-OT6*?SVhKAOsi*bN})5crMEg9fb<{*;sS3w7X%IH4O}* zx?C&?L+Dg48}6COA}H6z(qp?LJ0CetaF~J#llKu1^AVA0E_bn4xR>sR2+GQ58W>C6C@ z6{MnnWDxT+a2RL%9h7+!%j%qiX!OHQZejTjzWIb#AYXbb95`WWY^9J0b^p1Hp1X8M1oi4l?l&k_y6KU?vEIR0GeDX%r;@^SMyNx#A482fE%yrzf6_X(M*bQ!q zN~(_(jPxNUAdVI9u5xf*rr>F4Ndc)7$28h37{~_5{!bx-pHAiBbSjm041&hzTY6wH zsN9Cq2V;x(fe0z*ms97#@OQp-Fm^&vGSv)W6kNO5>Z8}OCI~DVqPV4fSO;L_bU+Vn zlZ@9u9}ol{*H<>w!3#eeV{gC&)mfSdzMNO~$$5`?hjBe%|t{%Pk5t zE~jAw@j}$)TUat>yr+iJ%v3!B3J#{pcyA?D4{ZbD`r>?@Z!&1kt?akRbd)W}<6T?# z;q<*^Z_mGgni3!aJ5OggGBJ3| zrOuvQ7ai>G8ZMPx4Z{!(*|6?pa!_Rn7B=Voz3J#^7SXfR`FwBVJW|iL~j1i^pv2y)}96x$HGse9cxn|)fh`A-uLiK)5 z=ot4B#B)@wG!Gd;-a5qH(-Q9?p&)OJdk*5C!|&v5m@c5!lhdUp(Y&4<%^3Ggq>T+* z;O#v>t3pBb828;s9EP9Ov)8D;3vq*5;#;UL0CME>TGD(hc%mO-+*1(Wt0k>fKhW9R zpa3j&F6&1D<5@$60|uF8pLxA$ysrj!59Yx21nPX{FUtZ%Rd8VbJOH%lh4 zD8*j7tt5js+{GSJ^2C3n+39X}zoKCXRyph2n;Qz)RGEiL5u$)eEVCsra1y%-=PD~F zvC&psL6Ui568l;dE_F&t=KgyT3c@sxEmnJB0o4Hevrbewg)P!{BRZ#l2g;ht4&Yjk zJdM4piLB)vX#X^}99x{B(;+S61&QfwReT{fY|h%=#@RrNf?g_#_*?r?#{Dcz86nfa z`&nF6nQR#rP8G&GU3NqC%@-v>F*G$}_A>4{T#8#-ONKT=Vv)OdNyD=F25%j~)x@GnAY_>>& znt1A??2V|#{Q3lgzpUbZTq;|LnLi)Kcsv|N-5NR=HSUSJm#QXlTxeJ9fRTc~`WjFaX$STje{k~yruE&2&u9tc*P^xGV0gwH=t zQFB?Wf35DH-^5ihZ}TN;z+A`|K6LL~mK}8-s1zvDPQDX<4N&XjxmTYs_ivAzZ_Q=v zy!_~(ar4+}i}zek`fDDW6`B4Rmf+I^u#|+Ju0DyEQL)JJqAq%TKKn;@(cH&bRzwym zzye(x0t|FuumKB?z~Y0)*^G9zvnRo<1mZcL6P2`*QT}w(6YSB@TGzWbH^qlYXZJlW zjd4UuJ1aTjwNF4og^n=L@ErnTB>ApSDU=L1l7>Jvnk)OBc7;o0c$oYau;{+xtdbM> zseegj*_sq0vgRdPSwpR?L#7UBnka{#F2ricH)Nc~0>nKFpiJR!9!|zZjGr2%3Ot|0FT#^)?nl2mR8t}a6Q7jSv>s}g!d}pr|VT|%ue7|rbOW^{aX2# zZYaO*hVpCXOAFB3A*UhJSOD^|BWeZUnHvB+bpwDW5f)%7f-W@QIf4 zR`_2xlz()70!*xwn9qfXTV;8bJ(yR0o zi{ZHLPkI$QK1_1_NhWU2@P$8k&JAc!Caa###vL|jevaWG(T+zvxf_~PM$w*Q~o4{1mVUomz?UA5H$xu(%#o zW%~k+HR$!$#BMwH`rh_MxOHzcDn!{t7?s5vC8*{rjFNiIWk_5@j(;aiw-s0Ms0(vw z<1;KlX`W+#^bGseE^jw?tOEA(%~ZJ>yXPD_vKk6I2Q{t66mt1ddSVUhrJQ-xd}j@7 zr|{$W=OHA<);_|Iq>B%76)mQe+Wc}S$iuhe!S10nejPx_^vF8)6t`&OM#;f$j@6sw zzn-P~Vo%^eTsUrqk9Z2WY9Zmi(uzB(`wEP9O!e=@v;-wkIzQ`twH?gE# z1l8(6wc$JoMqYJ~!f{@LWEA6{PXxTL2O+I^p>BL8;;<=tn1iv>@{oqp7hJ5*hVi&= z)F2O>4Fi18L$fW-MK#N7s`1Tha#G)yFhgo|*Gte%zD7%4Vz&&wEMNt@3IAeq$jJ=y z{)^Y84X8ZlNq^Rp&Q&fVpQrxcJn2g*as!*~f4~!*g8y(@y8$O5GHRSCo0;6;|j8)ko6st!$;0R2*p@-wM8zpEq~AvDpH7iyfY| zIm|uU^nVRoh#>SDbQ~V5WjF!$r}o>}V=|s!+Q#T=!TvoRmRVkE2lv8)@uR`eS}gO93eGbnU|04P+sMQar?Ix% z1y2hz9aH<2buqVpLc2!ZLVgsL$N%UN*#j? z0i+*M!4C8rQ!~;_JVoi^wnH0V9$8_pxNX z8Z|cCY;{zgZ;PVa_OXYeenXVrws@3tg1&z1vAKtopwQ3xEcnZGI<}9EMHTg3fPbjC zgSUB_r56=or?fLfRmG?Z>d7yua6gtcyluIky$Z97oDZ=5pG7qvV8)zeUipBfDsfQ4 zW*DzdX|;{tFW0R?nCsg+(M=y?2tH&!@G(Rrewpu6wgBp>l26%_9U#$UimP{jA~#pf z{x=k&Dg86npC9Xg#+E9_r_$e_ft$ag?w{i{q}p8aIeS|U)PpUw%QMA!!D!eEgQAU7 z=+OhLv+~z{^a}pv6BPL+bHH>MlXflt?l12(@A{Gj%d%|F`Zo*Uqobsjy^hNZ1zjnt zjv302lj);6Ha@yGzc|CV2UF(a>aK=&|IO#VC+?%pH7w3Y5nK&r92Fj9`8YL(L(VM+ z*%W*^vMBfvn;&1BFLf0ITK2`rxEGi3_!L~6WmEvprF-f1Lo5nsE}tG^y)gg6hbJEq zJnbP0`igB={_Je-`-+vwAUTu)Kiwmhfnc@L^Wj$GtL>uC+EU>+kU>VaH_A8lsf5f| zs#k=FHxb-h;689s+@-BJf0*hyKHZv}TR8>HrI)D2a#VG4i&D9DKTpRuq`OlK?3IN` z=g0K*oA53=YCQ(DbUU|_*4?jn;)YnV+XqUq6xo$C1Q-n|j5CJvtpJz@K+eT@+8c<(Q{J~MIp>j1QrRk~tR4p0$~Ja0 z-h!#kd8iYGqPmEijEA_-Bym+%n0t0B&nLa8;#>9$Lr3IrS0vwIX|U6_@7N)^m`VzNNs^%*W^SA%7!;0Po&!VHkhv+~9 zo1@Ju@ChHnLLiZdN<@5lU!z=!TEXv+o92*+3Mbccqjw)rEIkE#uw+}x|AnRLB9nMk|PD>_E`WiLvfkcq6nwxaZn}! z7z;oJ*U~u8GE3-y+9U5n>8Pz9WF9v9=@(X7Dbq;L_^bP75FIoeD) z%|ethQ}D(n2Ka#_m8NUw4}9d7ghKB5f%pF~NyZrnC8go?@fntq+mvkW#_>jJcbpCz5y8^g>dVbzE&xR_swB*ZfyJZl zns`mb;Io!sI*4o;zq6BE6KOZLQ|uosCRmO)oKvSC?z=v$w5UjGO!1ktPH;kWgD+!O8=w60oQZr$kTTzMPqIL8LW zJRHV(si>SFb3m4r6P*nJrB#2I;y$A`ctR#ERZ^0V1xe~u+9I!z}9eJW&aJt z{r|rpVE282b)xg5?D5oi5%zcmq+Wsm(T`SMV&nWj?1tHbpOvC-bLs3QY*28r(8P-D zjfq~Hz#p4fXrLZ#aQE$;F2}$>ACdewi}7 zNg+q4NoOZPbo}H=IS9T&cjM$5T@>uDBLg6kP@pwn!rD;}%a;qdeI?cuDB*d`o+-us+^;K&Pv05Kp}2Dunp0H1{ei=D~k>a6Y9pv!kImFp}X2TDC5C zb*wf#Z9FQ<>7QWSf&V);MH$f`_E>!HV$nt)9F}FFDdP~n7#6*nv~)Ri^@m!2EUZ4V zHoC3A@;9r!za`K#SxZ$K5@?OAt>+kIDJbwd6)M`pJoqgSE+N&XJ>6qSn86|UfbjcG z7;3`*3tF_1jsJ`J2}Upcb8LFzKf;VQa`2xsJd8fJX{q|0XrmBiL2jEiT~4Qa?b>j- zKiXy2a=}{X?OJc;ZzpwCwM4~?rg5tFoL&x|L1R9nv#K`7Q4wXN0ALFGW`iP~b;!%s zbwsKUn(oB+%e0(nV>)8J<^8zQDV4eKEGX}x1dK@sN=jJe_{zh16%Ep~KDvibaX_bO zf{{cG7_-|ot*c+F{I}2*RUg-&gOfteXGyy`_9cMCJ=VZF1Q7HQ);zg@ zV%uo(nR)4$O6oPy6$>#~M3uOB*$-=h;tk(=Z5o+Q546$Vi)cZ!k;-e7Hj9%wPg9ke4@>%j2xk!~!YBf;8E9pjs0 z=AgA9Ep+7QSSz+dL801w9}lNB#L}8jEkkJtrjJ9lUVe~?y#27B{KK^8lN?q(vTTj6 zg}7-6R)_V_wS}yVJ`K|bb^7XB*DQ!dKL=SYI%aiAt<O>0Xo>p><4Z#nb7i!=+huY&_$^3Ur!6XTwpp6|{JIF+wZo>!H7ja^z5)$}8FZ8u(VCzJea+Lqv)JrOJTjp<^q(xI#6lgdO^;5)-l{;Avia*; z3sHMMWC79l@JY2j8Y0P9`Y>8c3T(ykd@aXXh*-xtFN24;`=A-sLbT@l6)uX0xU;Em zjF#+^jwI}>LL}M|t--HMjHZE^9X`frS{b7~?VaVWMpHtpmZ*99hl)S6$0)x=(=!377(MPe`B#0;}aaxxA84SX;XZ`!9futT0a1S-bX*1wsaB95PCkDH( z{t(2A%Cn6Dd;^60jZixX>!7=-CSL30cR5;kK-dq0W|~d$S`XOGEH3sjvz%I$-M4me zxeW$jvRgCHwAVsx3m|#}U= zdnXFli_DY#Rub2p_QPB zeK5YUqp*w|cwxRX<7=acFSr&z3Gv5~9^ftSPz=6<6Zy)y?t<^GLfi|Hk8gwcBO~XA z!%gKpm6}tuT{zX)m8uPDCx;oZp~D-Q{osO#t5Y*6AWiGg2k~2zTjGRKX(~2nL!2?V z4WLy1ikkyl7s0J7?hNGtWv)-*LIuCiv65z`VW8K?2qYFlV1fsoG%az+rBJ^88i)n< zW-P9Xc<2{=TP1agwK~M%>0y_z0Mr1@=NJ%_>wqAkR5|_P3~oB{ODJ7U(_(S!tOOUH z@kRhHJi`|Z*ZeMk2{Sma|8Pq+1=OJG2{d4+-$_e~pPyuG1U>eIT6!e`voy(=gjMBj z#8K%P9GiRxfHLkychWL!zYnv#{Mh8-9yRdHbS)U#p?cWTio=o+_n*V)xpb{7WXuoK zwQSgNUrE=7Xtk27opF2^4eP8$$#2rs&KkkYTF%fqGaR+FgNKgp8OS|>9>~z9r;I}@ zOCVF_;hqa`@iG|NSO*T))slT*T*bi%KbVobXn#T@Sj_9U4nx97&eVFe7476C;Db1V zZpzd;$5%;Gdu!}C0xVXSU>6F_MGdr$zXI<}49BCx>77jA{~n#m)Drkn!?m{k8@9sQ zWaY|mda|pQ54rr)u38o^ag_&qQC^l7E_b8xS=u<52khXX2>LWjOS6wa4_G&xU+AXA z+u{EznD%zl(u1~tewDo2d%hgLY+mW6^-?z_e;{buitMtxZzuLTS6_Ke#7` zbE&hs7k%7QbHIM3I7eu^QXy~zU(M)&{)7=nj={%9d&I4}l5NG+=7@fG@O)QttvCl< zh@s!3dA`d7v$;w}3~)yeP*!)#PKXKilMv={RkuoImuI|F`S)b=z8vi-9}R^{ z^Bq0Ri}~6}d4VU?0%GwH0fOfc8rfom1#qa}!c z*&AQb$BHixH7XGQk~jV?BClAzev=(AY67laCR zzjqztzJi%d#EJ4PJ-?ub==ugP09SMifY+`A&_)2*(h_&AqK)1-PmiqkhFftg^TyTl za@-44s9WZ7@gDx;A5VpJ<-Z?`1RH?}|lSgWBW(D*4ICJkPtw34O zxLDXNOU5zG`(7jR<2MQF!K%Kuk-!hCs$&GXAM?hedqlh=R&>uSZ#=q3#23ek{L@?F z`+4#^MDN_|jlZ;@ucv}|(LEEr3E*P^s2&xdVdJg%Ml7LLyem$`k4Ah)e%Jw3AHTTh z^>Of&H@*;XJ$?_CAWpz6h%ua)IZF_3rr6uGNA0!KYjTNh)1Kp8oilXtIf%EAzkRc&!8m#3hMln>(~Fa<@jkhQqP7H1=aJ94G8TTE*9RVbv$nj0xJ^$|K?Q z?+IF0Wo|gx?$qw!;fZ%@9eTCkn{b-aECh zwheybu2w2P9IDBo<9BL3;xcjJ6q3i*RZ?9FKba~?ML4%o(4nM*nSGb0$jZx3>U}p9 zN!8Ib`EKna9?qDk^$LTatbm~0WQR))GS_R|6GICoYDY9bEDulP45z3-JEi#}u&pM{ z99^h=DGw`1F%&EY{H`UIU9KLCDV)|ElPPlmR1yLH!EYDv|*|SyEiADF4F#`JhIQ6e2@0Jthh2L{60+> z^{u#1d&l>b!|>r7)bkD+K3VIf)H`U!WDxy|gZ56=1~_U{1A1I`EKb06Qh!&I$QKABtSNv#02G_5#c{NTPt}IC zM$??CJ?MMhgUwyYH%%L&6gX+#G_9wdV?pLL;Dwq_Ptz7E^PMzfI>P1Av}wBbgmMm; z+>aWHoiy=&ZN&8@PTUV1mpRQ_9?;s!N+tN{K`qXAbGYG)MhvM4r+|mGPx+!!DMuzNr3+(h}9SB;vPQ)z_Hv*9s_)Itre?%LSV2wP8Rqe&dE9eXBSadh6 ziRVI&uhN0ryN_u7;;fNam0#UnjKo36$Ion?TxOTj&G1LHuiDtFc72&-{!yaEDjJ3v ztU1ytc(L|7=ZWSe7VAbY(T=qahd+t*JDsF0MbCpbdN0+|(LFeV0DH_>ilGViSie;3 zkTBnA@W<>E@Z;kPuEn|+Tw_QGY|(UlsrDoXHhq}|Y{N1wvjy0<%Pe5F#WknSMv*OYMLnJk;VT@tg$v;PLU$1N3Wrw&v4{Q?cZO4HIw9P*60E}y8DS-1EV}rb)@4eS^#&oJU6&is7D)n%$X(IR;6?+#V&HLi8i0DOTq4hh{MVI7?V>z3 zoT9d_=Cj+hp0Z!<(;wy-XMof;wKr|wt~u%4>so(%?U0qZ)O))&&39$Gm3*c%PoAcv zccthm?VdhW^4xuW<;4}l^jqWGokINyDK}k_7H@3pQ`~=KifkX5AScHuF@D2b_J{{X z`s4fh^8dt=ee`aX)~!!%el#ZUaxKnWu?yBu zodjqvsOPRF3%q%ezoFyzfTi5GF-`IkH1TaM!up!`wziM2ow>WU3;aL(9ZYNZH$DH^ z?|QGM6>>-ab4y|49>`G>hz<4^qP_&iaq4Z!CRd}i#^ zuJX0p%=ke2o57yKO#58>m#n~L!Im$8$+i@8`T^}lSsA#>j61BgRYLy*1F%V0!ZyCl zH^2~Fu}VKr>2)|eqUY#8KWN{;c~HecZ6d6ynD;!zV! zlqYPmcF-KaNE-YN^iNe=Ui(Iqv8mnmEzX91Hm`iE;R5PBGxo3+poES8%BnebS5uBEC4^0UjtBAZUHE$ z*A^?wpRpS75$$O}1MV_okKyD`?rUZqhvql(i@Ii-f)F2JAr!z@(Ha-X|sXuJE8Sc;^xpRC$!GmCtRV^5Jf+n&|=&E_R>|7e6WcH0gk;y z!!B#R2uC>%c#bYSsr7CzGS>)2T2sTK#eKMq?a2(OX%8<4>q1`cO{zJm4U~T*pHtd! zZ6CnryZX|^Q(A1$3S{evG0#tM`L@28R-V!($m8hLDXn+Feq=3zb<+pZO=&|bBlB+a?miz`?%Y(G%H_fHw$<+Lt7VE$1MKlb~UU|dTQpy=ES{d^k z4NK6&f_6N|?NEc=GLpGmxSSq1qveLqMG20i9jATgz*9U6G>~V|Ypcw!&uF7%_@7Mq z19~I>F)vtIIq)Ioy?<(WI-(; zX{0gWN^|MrTD#GJxzYw{Hi~nQmiNYPsyV8;sOh|x6}$F8Gpz^l&I5|zZ$K3&qr#Eq z;0xMt#U6k>6ez1P^!6pprdx5auKk0@3(hp56UR~azqLeI@Qwdlg9RV0{#(nkIHm+I zh5n5$%~%HlwXq1)9^ds;@sE~HssCslSFrxqWYp*e4Ee*X4E?GvEm$iYBl}Pb2#H;xBGDL-GT-5^QgzCP=Us0`i0pULo zZsCRMzSR4w);DrsD>#E&!O7)t+R7==)cP96HAo`7@){%&UXC!iHZq4Ux*GB$H8w+C z-MBhQUnYONYn#4EmPb&gqWAZIZrfD~_n5Twp%O)(=NTMN>&Q(Joz|(zzhRGfLWt{y)Cox`F(C1ehFtNfu(I=K_fs zNm3Bq;-e?|L1)*)iYW-9c|LkPc`UbHQ11YgHFg6H!Z&9dIE*O!2u$gT zr*D1q6!^E7ef1u474`Jh8-3n*1wC>cMiZ3TM$b|rAEf)*=yCE-w4{xm=HZ-n6i_aQ z28i0d4XJKNKeW*!0p@%geHg4QL4=NUzn@+&cc!8K`e-F`K5g{ZZ-%$WQ~vt)7Se5K ztA|=}ox#>%uG(^y*2624z6peHpM&`3(NZPK)Jk)*xWCWFqWVzdW&AI$52vDbdY9BH z-Dx*Y#lxQfhBx<+SqaZ-zryE(fCTlFlm{ZCzt+;8c6x492Vm7n%{oVn%@sNiQNQk4c9vb{qdZKY@h(26;IR8;d%mh zp%o*#CoQM)DxJr#N6_^X@}Mj5ELzyxC{N6-Tm~77A2MWgz1asR7Qss>Hd+tYszs#=^^Vrh z%1h1k7#*we%Kg@Aei8HS2!j*C5=L&ON8&Je|4sh!`nO7-{ahI6MC}}USHDB0SIN5C zWOI|_4;+m7?577Eda7rzgi%0}9%omYX%*Et^zdvkn>IumQ!)RiKqVvI_=fW%JrCY| zU>oMX0`GT5CvjL)j<@!#{LvKP#6YP5*3R-6N^|NC(6pu_4M()U0VE`-FFCI-rCCnB zw>+1&Il(u`lWeVog*=aEB^*bY33{~BZi!e4+s#;V{Yp5nlpZ{zb)Xj#^oW#fl;d;s zzTj)?;jHhjl8SZdfp*?F*%(Pj67+#lG05&^x@5{~f7G);s)BX293*2#V$J@7?DwFgHkz;Am)`$POQcWR>rpy} zcSqy43s-6H{d!DCs5CnoLlJ5%a-8HGUK8r&aB++V4(}AvsOM!HgOO_Wr{I5YYN%EL zES$yJ^lS$`-Hw4)O4S|o2>WUN^8#IC{BEW{JL`|iznfDt zbQ~arkD-S$^*ek7$4jZkQu;bm&y|0t(5^bJyE_pzg+{h~kEfMg^?r!2?`ULGT~~dD z;{P<=ou%i?m#HF450@{{?kv>)I~~r_lW~7a?uHS)m)3UEbAz|9xJvwrV>P^k3#KVg zq0`;4oNXetyPg48lij=Py^|e*%_O>{P#4qD*5oXlrxuUe+uRLLVuoB2T~e-_)LkDW zcLhhK2Vt6GnGMBPtIve6Ai*YKnGR2{k!-8_-4kCY?VJav_( zz=&Wgr0@dV@t04{JH%)AQW}|~KPzvbh8#UHa$tYdmM9xPE#lV?xupPCMnt(GyY#|% zQTx%|z4Y6Z=>GIpF9-!6(bv6D%J+YiDyN{{`XRZ2e(tRg5B_dBmJKu%dti&ij*c|2 zkDdcJ4kdl`Xk0{kv5!7IcoVX5Nw?nH6U7}Vy07l!t9oBOO@4}|_QlL4htZRL^@Q=O zz<8~Dg?L(#tB*|lYuQ!0Qviq(08S#@1W~7T#^zkTF!3M~ zEQrwp;>QU81;=2mGcI!JbL_R-@*=6(rH`@K=2u0TQ2U9@L}9Ia$ z&nb~3Y1&9V6Q}#-4x_wCsar*xN9y_9y}46_i|Oo0y@S0rr^rdsqx6*!dn!j^2oIvV zQF^wLu$X*D>z#wcfeIJm9YKP$P=-SPJX*g?hEw~>G5V``?rH2;Jv(^eqN~JxpcGhL z09{D}Z5WHO{wVDp3)Y(P|44iLu(*oz0ekiXID0~pO;{6>V6sM%U?c{WM3N9CiUyS= zL{KrJArTEKCR9;T(M6?-N)1tTXho$J6_qMlTB%ZlmA0s;sHn73#g-~6DpoY8G%*>hdJhNwS*84YEbNQm&*JrY9Y}cN%ta*o?ORC{>GrsTc z<6ZY#GkEWXb%%pr5R}1^r`MfjJu6=eKg&8iusJ$7BwMY}A~DQY&&{4?+GE4JfJs<(z2xI(bvcIbNZfa}b;Po7ZrZQq{cDT(^M{t0!3&&wJ_x-@QD6C-&*Y9B23v6eJ>BF9M(#k`!kY+_XfKqc}N%V)0-l?0IzO8$9gHT^+YUr ze{-3~z+h|fzl&cU14DlO;1*u4a!P)vyHTgI=ArUBtnf(v{ucJ%o!qAGJ(nPH$Q^99 zr&VrWY#Ca0`ngtG?EPB~1jkG;Lgl&Eq0U;Te(+o?6Q$O+eN z%hqBFGS`d(3xWlzWWA=qS~zpFYmaR6AA1PHmR!|eChJ`V7C(J$ty>n{(=7RN`+qY# zK+p2hBfiTw=~rPT*^SG>E$LjQ4-~LQChJw_Sqm1-t_u=!c!xv;y;9d&k%QwL@mvU2(3m7i&lk=@3-2NN)&kp|1bkbI9Hf`m^h-^xy z`6w*BDYb9Z7j3mx%e{CjyHdJlO>kE_ZK(3UyVAc^>ZMve4p1uGTsg>`DJe+y2{=|7Oa`*YUzHB@13o?=~yxu-h@={)5ThE&aOAgOa-P zF3WMmxWtfmtnSn2Y_pDBu!m@a)79^Xk41|v+H+nL-OOvSX=ks+`-UFcW*s)QMM~sF z$nqw#Kj=5NS%)9ny`OzMuh5b=a6Wc8U$&F@H^v1w2cJEp9{cow`EuQ-4>=yAe>uPu zrECX*&gE+qGcK^MuC0W|3SXs<7*Dg4tG9XT`ONMwD=jDmnE;K7wV^W5Vv1{w8NT_DUecciI^b+f+!v^OEf6)HfNu%268+dI1Pv)Hm z^a~{xhvQD{JVHNT!gCW9SgJoPvEqH)<5%zTBwYWtb@9c#;0&YtjZ^iM%dGi}sm3sG z&J}A-aFlOz}dA4A(|rO0(X$}Dr7X^>dCM+G)+1iMc7fxQ2z(cLg)Laitj`1-rOk_v*@zqvCbxEVolL)?L2i+jCiMFhRxEtA~mPjYXc3Es1C!jpn&f`OxC318 zUhm_R1W!(KKQqbwyGicnCb?gj3AT_Xm^Q z|Cr?dc#`|GN$xKvx&J-M{oN$@4`FwJ*6$B{a68xiw?e$jBQ&aey}Z*S;+{6iJ#&(K z)+G1bN$&ZR+zThU7vm0?Dc9@eO}3E&j+*2?W|Dj5B=_n`?&Bx9Pn_gF#kim55+D7K z6?;b4?|rOPE_bwPaE?;(&!%2!wI1^9#O>D04*h10^gv4nTI#692&B-gk@>4O;@n_zXh9O>?Emnuzb z3+95e+mY7kQq|5Z+`~=_ca19^cbhAr&lOJo|0#Xo|8EBWo4(HBxYLnhb;de!++DG% z%8}L@t9l*c?suehpoDRHad$Zzo!zl&-0cqFtBbObG(^=q(2hhW)RQ7QlZ|zbo#9o~s})#$Krw>#X`bJU0< zZ8ToBJH_4aOsk)xT3l&0bJUPa;#9dU32}Qx8s)2-s|KQdqjS_~Gh8ynvrud2QMPUI6a(>mf+TU;#eCSQ6>yc+U(8|J9iL(=Nv z)z~2tUNuYdXqc6bdo<{-pY0xuSDmvZ=J0HHPrRC#?dzrm@r}-|x#510fqJo{$Zs`y z92suixW~FEu*$)>8YSy<9Q`hWYAtWaEY)Fodk;~S9v>mip7=g2Id*yt zFDdMs65lvWjZCRfYB<*0bcpJjDgjl~ybZHe?I6by$mYSHJM%s37 z8gGvrnjvvIXU6i%?w*-m;!Mnxg8SmqtNAK~&qCVZvxwj9^AW$x=fgeVizR$C=&p`A zq`}!S+Yxwn)7{pL8eXHKp53~@^Kz7RAoS5mi#$dAemQ*?4%wXK;b7S!&;8ETe*No3 zp1Hbeu_tXt(S_yyl3f>E>Mz}O@fEvvURb1`TkJVmAHBqLwtjGl=b~dz*>-Wsg+>0d zUH;22Tz2u!D=xge%zxE|7nJR~-0v^C^74y!?(i2~Na)4eN-n;3TiM0CcCOYrNuEOe zWRmA{opgk!^V!iOJOh^n3TGr<0rnD(F0&9P#RN58r zIpnZ@ls)rRmTKRx)Wp@6Dvffeq;gBKNUI#!AMH?;xGq^b{vIioJozp#!|hU@hgSn(|K-#EnbHjgefzaf7e=S7$!n1j_*^##6uja3|7QD zR6R`c^QPVFES1WeNGbz3G8Z{iCrnsE1{L%`66*@=SxN+$acG*0qaW53 zIMf8pIp3kOZlnc;4pjsDwjqa|7tn<_St{cqhbo7SunyK=>`>h>;Szd6{FgeEb+e_i zcM%_sasRA>9hW1AnO9H_beFN3SXGv4D&w1xaPUf64m+-LsA5=tjYIXo%vut_j6Dt& z`%^N$%b`+W^W9Vgmfg>#gM$xK5t#Y7LwRqZ;NLq`AxwPUp(@~LAC~}Tzw1z4u&qBp z*W79;-}|%#CVuEp?XYFgp@v}hM-*7iVENdg{IKm41`qTPJ5&MefyJ;6TS_(T_>2m` zQp`f*FlmHKcN=5u3x~>v)|U=d0?WRlXJKPtlp5j4{@S6Mp!zps0d{}mP$RJRI|kkD zmWrJq16b{Fs!BKzjedWg_*fj3^u@FI0naHZ=O>n)WS2Jj2{MNzEhRJoO3v6 z3gZi$s=Joyf%a1k5KyrL1u3s+N#IQ)39t*MLF-bdDuC6cv&7-l?kK0IU`H&yn9v&)nx!Ww4@wOBcXV^Z+e_-Un$B?0^HXw~-dX z#$P(s1T1Z$>-ffF)gu%Lvmc{nu#Yb%mcr2|s2EK6HC+$8VLL2)lKi3nDe{M-a0m`Q z?NkBxJuH#GAwyWy#-)Izzjdl?SpPd}3X9vx7`FbNjA6}lTuWHqL66-_0Vo&!u;q`m z5Z1oLpo61-a;joD{AZ`ChM6zZ?OTP^Saa! zOrPOWu@BIKIF~Ae!>|@k_}Dx__bf7i*>FfWoA?hhSUIAWB%I??1+Ws9L-$;ls)u9T z<_BP1f=f-nK>VRDmC(o_gXyqmJ_W+T1r!KN{hWD&eXs*29L945tb~2AXBj_7{Y$!Z zxl6f!WvP)A@_{95T*~_pqyGf*fhn0TH6lF4rIMRy@To49)5QGmK8+d@ka0SXBd|G} z#~PS(hD)VAOa^&`!@4tFss(z_q6c6;?1CMz7p9#}%iuV4KSG7`T`Cn;p2H=8WY9fv}*`#Y=2yz>V}2Y=rr+>Lyw!;Z=+&nEX=;gc-MR$>1m)f=#!& zRA3Cp@a=TX6Wla@Mgedf_QIsyE*2GP4pj?X%(tGCVMZMp!}@z%st4Bmf|kSh`x#5W zX7D~hg<#h&=`q;y5Iq1>9(O70Nv19I!M-P40p-V0*Wyx{Fz?qcRRrBn@qjDgPg7%< z-$uqy;r|^y1IM1{p%wBPf0fZnMRTeMUv#$mh&s82^P! zRlq*j0DWIE9br34fN^MjLxr9t{Qa01p%cdPj48AM*UN`(`!0QSvrt14JE z)2$j|>LG5`4J&53)etP2?N;s%dM=)VVbdJ9%7ATi-Kqi>&U343k;7()4Mh!iF#n7E zB=`f@`fzFnI}+Wh3+69&t8tjU&aM0}us2-qR@t!ORC)rIpH7Qlb+%ho!p03$45n?Q zXJEo68U{<@2+YW#XZDiMX17X&6)+=!BkK%W2s>dhEYEdwOQoi;61L^JRRe53)2-TJ z<5_f__|JB$QJ9zSRV)LDh*~9xK$?1JD+@E+6CkbdtfcBFLJ9u8;+3+ zsWBYfL58s55-f=?GF?i@7?xgx|Bvu`xAMY?DtZK_-@+w=jkmg07Ifc6Lty6Z^bD-J zgG&aJTIg9=(B@{3z`db^@?iT*Zl2k3)W5>DfTeFT24M3S6!a3)=_?9=<6pZ~J}mgg ztx90Sk6cTkBTBWv5+`~#=ypY^0hsPazXo$=M5)*=#?FaRDgoBr9Hkmz^&WI?u#v|;V9JvGe09E*zpw^KYa-v}%NzheWGRSTjFbrMyA` z{%DmC3lgJM1squvtp+4KDO&kn<(_gxv`U4QOQTf|>|YkG%3;pY(W(a49LtMPUSM=!dnfWCVwwj#inl?KiX(cK_L4F5cSfr|*Z~J&>%15>4#yH=RP0-%n;)YRq0b+q(qQd7F}$(8 zij3Zk!2rzkgLyw?^#6@-#NNRm{3J#-?V$^Yk;9}h(R=$o=qp)uu z>F#C>OvI=LSQgE|W1BuPjZI236`Ic40yAdN+fP&RnU*>u2}i;lMkMpReG!WVOi89t zIB*)3Vz7;7lL3RJX#*L+#!a-6Yh9Z|=3KgwGw60MO|4*yeB5(UP)+D8RSq>a zMY)O{Xf~8`*2A1w<*INRe}l`R@~0`+m>ZP~OhPv!@kCdX>{YpVpu_RbRJo$VL4%b$HpN5~T~0fG(dAfiDpzzl zSx|I2^-y#<{9aqG=yEcADpzzl^{^ZIrz=;MkBY%YC>3gjjrb=*{=`F2ER2$%XckH& z1fu-OI7H=&@~0k(@@E)|?k8oI$`##DB^2FHFBIL6e>N348Vv^&-A_9d-H$gO`AH}q zpy+uIPlapy-6^VcUhMXb$C4!j$>S6-c_sbd4yB z(hsACuo8;0s1u5^$Xcj!>rfU+7mCs-AAd>-6(WOT*a%19FzmU^#LL=A0SlC?5K6js zScqSgP3@!?Ws`Nd3gkARZITG0b;>`SuD*hr!UibeXrEL+{sJg|Q9{Khs$5Y*l|oTM zwZrzSjl2``!x^s3a^m@ws~bwX9+-uH2$sWGlvb_&fO3iUs&Wxsb1exGT(Tv~C0eg8{5`*b z_(l7bkW9rNM#}|7`_&h~5&sn0t0d*>d!2u{O6g67rGRDsHq=&;$!o3|R8j08Sg7J^QhNU!QFN#I~QM9Wl+>`!*EQvoR$utKul4tg1<8+p!Wmxe6SCuAER8NuFFVe zunwZWgQBh*fEoWVW(sHJKPZ@pT~KN~3LEgp5209u1yI6E-~|4`kU!~P6mTpRg(a~4 zV-rvGfC($8@F#>r(F1ls(F6KcGE2|{N`7^pGXIMR5QSh96oudz6op{+aVl37f=y5q zf)h{_f_Z6Vh(54^0vllqe$flMSE*dl3+4&GF$JW;5#ouSu!(Td6Z%%GT+tI&!Rmkm z9Se?pzEYaMM!9-jXeOvxtlOzZ@K?Zubf!%dx*nJw?Nk-83rajuFlHT3h87wT*b7^b zk9g2@Kx>ML7Y}3CGCik;(oIKOhCd)0N$&|Pt#KqIAoCDZDo`|(`GgnGMiY60a@EA6 zv4mq#3Kq>}=Q?@_suNYNXfAW0Xf7M!2=r%guSh^;14WZLO1LO8Q%+)GLy=i|67yd) znY{#vCNm|In+U8WfoL>q@i)V6{G!<$#V?x8oRe9Y7E|$ctmRNDB%01z!bQ_L3`NtK zbqdoAO=mL{O{ZEMLjQMC?1QMD%K(BfRwb5K;R zO;A*=Lr_$$$(!Ms=nG-%Stu5z>)LP>O2)7jj=^qNbT%!9y)bDLEy-t0K+(|-oxwCZ z2i0pXm+V~PZ)Um{pn$|b1|?tXJkk?h1e4C7#jqGQo=-eDdOnK9iChZYis~~@xl*^8 zggMZECf9ts@lU`K{GtO+Ig8u%MJ8Ny!M?K@e8qI}S;{3k;Zppf6K;p16ZYma*h)p|b`K5-!^Bsue?y@irTyfirPH> zVy4-BGz5y;d;$(ZX}}09-$8!&8-E#W*`Wfibkyn+AsY@5LDcNom(axzGJ2t?*(YFe zBP&=jgArnE%oTNe6RiFf6(GC|N(6iC9LW`aLZnyx=(# zF8Y1%l`KRZOiNhsf(aKrf6CR|Y&w}$S2As1G~vZ?48Q3AE0E{E#Qaacia~>epGL|p z>@pF{VFTe^e`20PF%V>wvt<66*#cX-O}u{LHH`k3S@Eu=CtpDgkG~q0!G0(W5JN)j zbxccG0>zHd0kI=sSg2q#i6Nl`w!sz>ib0_RzaN8wWFUrxgzH%m-!y(PFjU|d149=S z1B34dw$tx0hDa}lh9dl(e>3re1_%7{16<;3l}ii{=rnTs|H1StAwVn;{1Q&CSRk5Y zOngjN!Lm>2+DcYNSOXJ=DG-WfBIPD3^eM9dieaJ~ieVz@W-j3`(@id6(~v>*@*#vp=}Z;c=o4iN7CF6eh>8ai`A{wA1O#oZ6){gj^A$Ne9Q z#Y5df&wbBb4vNL29*V_d7>-RaeM#4_pLGHT!~~LZD>n-sN=l)aKssRwCJ^a*F@<=m zSr0IURKosfRIMcRViB=^%C(2xgo|Y)={9-}%Sig|Tna2B^-wG$!%!?ENq0~YA4i*D z%ONN*Z)5%kO(X<}i6p)T&PFW`#YECCoa0jbc6P3qNGhS2NP404i=_@Yqf`;V7)7g@?IRBNUU#C=`=P)*gCd zIW2)=G8q*A8p7{l`#dK0t$2tJB?`9UQCtp~dWhyErmBM>i zKTe|u0yxB^G6KbCsUuuWEMrhiESZ|YibkHr3Z?MCE@2Z>>RY;P)sa6F!5Xr3(|FBYf1bCU5~A$0*a}n2U>-w(eESS zc2f~Cxs>5gxxo0dpzj{06YRc^N2^fyD4f7ArkVEp=}BlkKsrn^`A|$V%@ETJ6uSUT|cUxEPvip{2x@SMx&0sL({iGM#&KClRiEoTt9*mBYu$@p># zc!)s_2OD|7xq`9rOJ-A`B?4wEqbDwoQ@UkNMW2yBO9 z6smZbF29Ds3B@Lq^a#9;Q4Pf=G%Um>l<_E+_IlG(LE8|1u?-DEu??j(vt$P#-dIr#-jKqm~OW-i=fzw zx}n&L;#Qp(dTJa7#EFqAPS#->Wb>mOvSJ;es5k+Jj?^Ixn_od{wd z5@Qi}CulvxJb#2igukPiYuw8H`!Pm4Y=Sk$*+2ShdR9xe37*C|0cz;VWFi-*Z!g4N$CFBT%eb>CbV=U`+srShWVAShbR$XJ_&T z6@g;a8h~QeO71`oYoJ)Q2B28AlK((O`iKw3s?`C-s^xouK?w7pShZSU=YXlOShw7J z3IB-spNFILpS1W{2GbBV#V=N_F(_89^iD?WXC|RoyK3-@wW|+WUzl*QdJVnE^n)pX zr0c(A^uMTFV*M(|FV?SKSnzLV3E^S|OYfor-!lKxUt%zQ%cz867OO@eX0dK4X0iA` zakGHMP|RZ8P|RXUf2Ja^8j4kH7#2>L3W;?r>n}_2c}{c!y)E1Uk@#U1yIauO;F5hBT&q1sjtz+ zGcc1uF|T#O>^M^~F|)%$ze7$rL zEP`Th>wwvq+a#fw-D2Nlj2v$KVt&iRFXp!f=vy2NcZ(Tr41eYl<8Oh*Z*i@YOgjG& zZdHT729CqVrS#<6%>ULSiTE~}jia~(Fn_s8FbYTT_Z>sa`&jc=U=D+YXl<( z!o?678*n&sy&Fw{SR}LYi$$^libZk&W?+#_cRF&#BG~}NA~^uXAercL?7%OE$ym1|R}7OSPz;kjPz;mFQI6aR80aN|SSU*f5DR4+?1TQl@~CtUw%TY% z?jS70U%tguv;{Wf7Yk+{@^PsCMh{@gY{DOZ9+pk~O|TLsVcC=xi-mIxx!Q__l<@qm zY{JKJbP*sC24Keftm%cup9M?tCvRgk!w#qhl*_-J0-%^mvtt~&?NHK-$+Q8#m`sOY z;)QNiPP`J>48?+)YSF-pg7MwH9n618n6!gYNdz&imJy-77*pxrMWH|oU?c2>-b+k; zF}-5F$ejrJlP+Z)`G5zP5~i8Qky{QW-%8j>c+O=QnW29tJ^LXQfrYR=u!{&1P>M}g z0)E2a8D!L7PR(J{6(*y!E9o-)?N~hgapa0ExC)9bcmxK-7@QaD$Q5I72NYv);#5a&R}Hg)gt2!~bC|8E*gv?&_tPW7 zU(q8l=}`*&Cv0JUlTM7sBT$UT+0*FBmza)FY{#+FsqmYOHQ4l5F6}>=|Jm<{S~$SG zCZqblGp~o>2V4T^9irvX8aDC#FzH`hs?Uvo6c*!8`hsf=#VFlF0b-O+^fEZVrl+77 zrTbvnw-~XBSGv0m2^F4pTIDAw!L zne-4$`-FwdjZqnYNmP`o#NQ6bVMR3ZVdODUsvIhdZhO$O!LqErsyX~o7rQWAwVlbzBo zDgxaVCfoS1Dc?m4{#sCJa46< z$YY--Uswy}lFGS(666KXP}8J;Gru`N&_aM5AQ*!d2MB@@VLoy>P>?x`rMaCh#4iU6 z+VIQ4f>?g zg9x_#gLp9ZV+IjSg=!yzahPvXkdlbfx)ABDyK!{v`zwUi=k%1RK9K`5FDd zc87dRB8_x%^kPinarB}bzZ}6BI5gnM9r>QoNcXFFA~oN z(=QWHPQT=(Fzwp8gvjL-OcVaz=P57Q!Gj_dhJh3u9XQJ17;J~h%N#E24`cw<3uFx2 zp|q?64&fgQ`Q=#61R2Y*n#`ja?XU_a{xKNeT>=M>cDR!MWYRZ4-!TrC_s`^ij3eOc z#~~RH!zu(>f1yIKzMH{wEM51qk(a|{{9UjBdS5f)iLeQOYCjc$Wdn@%REMkl??&DV z8}Ju>VEox1a{dSZ5S01n%owDg6%JQF%!aZ5q$02ccEJ20^QX zJVq763fK%=5`zV}{fkUNnJ_1Vo>~&4s$tr(F{&TBkB?DF>!{#KF{%>gtdHSfIgZRz zV^sW!OsCUK2JJ8hf6D1Zgh?Arcn<8u-?z#5yK*?Yg}--m3`S+*hHgO&KxE+-wduAt&DHc%F$vQDL@uo`w; zNzGv0)f9Z1!!-cqT8_X%{H50!e-&)UpIkvkF#dW@tDVjuh7v#TMk<8Aq}up>chGbA zd+(;f*$i53!UJRVL5I8WJ_Z2l8t2&z{JhgL7))42I?p09ZUdqFs! z7(iXldA_to7zSY3k*LYu3ufr{^~b2>rEIxi5v+yH%eim=m2-V5Y_1E#|p(dQ-JDam_c(c_jyg$=Osvlvx!J}vt^MvXvggq|s67Y%*esOT5)B6L7sQz0k^4&6IY z!+%fwB5vCgrs8q{G0=7~+N=FBD!rJE0|u1_SO--JdprlH6Je%{gQhUqZ7Kg{bYT>y zlVK<1i*ufoR($G13OG|TwQVc$DE8$#<{luffu zcx*f;BR^nM$_dI69?$a^neg^S0a`|Y9HmU;F}mjzBN&D4Jj0c)r{+9zm1LRl7Ff=M zgqI_iQUN)7*~0F&@pR6Ivq{b52xcXeqnO!;vJv4ZW*w~FXyW(4O7?!^A%A0lGn!-U z$*MS`DH+OH%@(#w&79Q~zns||!#|K0^g9dAG~taff2qUO11pl5mS>spN?6Vko^mz^ zNo8q=lFy)_b13AOGoV#Dj$G$xzTWtbXR*_pr9b$)=NM;GwtoB_&q~)o_ILWh<@Pe| zeb-|-tqpqayPi{AnGb%ar`uB&>Cq27afyGro>PwA*aa$sBK-WNujhC1_=piwKGqdY zdhffQqvJ+TH3AL^8o>ok`lDOOMEZ34BhshqW)|IlP+R_~ul>+-N?gf<2Q}lphxNGwG$Zz*gPMWh`2o+F zGh6zNV0db%eSguv81yXHoB!@v;!FP62qSIW_1NS#)_h{xnDnV>Vf? z#lA@c>O9VL1_xBQjqRZ_B5geIXt0fONuQd+BiHNgN9B5*9GCLB5k#)lUmi7$?ETUR zBG(FmG%~I)v$9_HU zE6gKeKO zB^h@gfBh%z)4Ox*MZo|!L(iLOdUs|z(o_QKPBw+aJN5plQ|9S={^41i_%Dv;{?9ZH zSEYtj^07XDM1S}Xrg2TEjsqrKS#La|r^nhWbmunP9WV1ZeuXI@BOK!?UUBXnv=_!H zCDk6|FDn$MM4vO{@$0o8dZy_E*V;35#wQ;4>`bcJ8;a2q&SgDAJJw!%@4tx9K!n7V zrkE7m^09t=SbycRHw4!cH{CiCiXh!2AL}V1WZ0``A@d(+5_6E)$O6cw*=x1;W3rbC zUzuiN)Z>b5!}_UFV(gr1&zzl(xv6!Hi6Olr zABz~|F-!k?D$Cla6jehsU%H7FspxURimsY1#=5MKu-#;#tQhjhvJclceBzluJC)_A zITXVmuE*O=yhzbrHyib>DqU;x?hGe?A}E}vKls7p)yu2s^64LYqIemf?T$-5!9>mo z=fv7gm(J0B!ybP;sza52qKUzVE%>q4n-zAkKIT)}qe;;pu6Q7L>+m}YbZw38(;1)9)n!C%*mO|DUH?bKE+QswJ}BbW$r0n@ zeRM#}875AVJVo-cHibK5;pd)N1v24Eh}NEOqD7|C-A((0t4-##oUGVGy$?9^t1aMlRhiF0$&j-#yRHD+xk5|J@~oD9nZ#0ja_J>c5~(BW4RvQ zuScV%EY$Ny7%c@vDZ0c&kr@(d^}%h$B3+8q*K=(sZTR}Hkvn@on}gP` zJ@Ij4*BNn7SR5A|jSKV-Uwe*8D0IcCd8&7#FJjmVJ87Vp`BK z=81K7-J)L^W3$&?6S{oC&0cqnUj41-6p#C7CZ|Dt*t?!N`d8n27Q5o_Q|Jf=@O#y(1yen;Toz53ek$TV5+(Uqe zGGiZU3ba0LpXVrNU%kF|pXby?@0@qgEy-PF>QWf)>mwWY-z(QRcH9#i=MPCE1H{!R zYwaSPGtL0%zs01j3a8!2PyVzXu#^%tUq$B=KfjMt266Qw?! z%YzT<1-qz0Nr9c9GtRKx`os7sQNi0&-5n-wb2z)L59;NbD|FNMT%nPggQS;+q;H9| z=uRW84wrK^<I zTov~%CX^5758&}`Yihh-NA3l#Q7gNNcv5^NSEos_$lma zq^6&l4j;rVAM1^Jy})U&jT;IjHEKKr57p~kPW$Az{8p1^3HO9xitFn2U!3-axT1C= zD3a^Q$68ab11>fur@7ebm3A0uUs!r%P#If(*|tromrg%3uDr(xx=eO_0b848 zIAsNQaDH&!a_02v=iK&b&Xl)wbdNcu#-zJk?oJcg_I&W2ER!jClGP<4vA?((s&uG7mc zdtF@O$EKc8Sp|V;}Dv8P)S`{8eA;jY0k>PY@fOS^M?tw!JR#&C?F5?a-&D*y?cS?YK*3 z;3{2s3S%z*b0bX%OHa5@pd_5)?LB&{2#P`i8OidocJGmAf|KLgLxL(3uB?;x=v~K}y@Ok4Ol5DI@^6!A zWVaqi+R$!&R$TlyMjF|vBMtA=m&8?sr0wDCzp0(H2Tx`R90cF0vx{;Gt8zbV^GoiVBx_?rzg5=X>qNaf#oX4CIKve5~Vk%VsBB+Zpf?kt=9p0UK*H5TW!c1nMiq@-V z+FP8Bf7jREX)o2CGwoQXKSsYklfhaTqxTEhU>y|)J1u>_FmZ~$Q|OJ=F9>6&>M!FM z%#}0r@jjS+h`v*pHA{csvsWd)oDmw_MbyF1UtNr;Q)D?>bB8QPn`g_tx4<&O$ddHc z?ZG7}c>fwELgI{2gm4c%al77oEH}{kv+TKX?pa0}S*&lpU6$ihoo%!9y|e7qo`%^b zLb3iy;*>-5m$U4d&YrpY&U|~Wu8p#z^lfwPsrsnd_Domz+;RQjZS17RC^l;$9#PCS z&Sq5{T&N$PZJ*+aO*FZM_G&ZZ?W338zc$p~ERy;8OYJu|08tId$J$+O9!B)Kx9nMZ z7ti1eBwS{%%)g9K8zjAatgEZ_?w$4#!MjOTNH}U_OuK3`eC#<4pW%=+a_62Mlpd`! zV(r+tzEImE%Yv&qvfVci__gt-J&_&V8@HMr9b0~P`~VHenrC7}wrnrm`rj>^H>e0m z=E{^Nm}rIJYTkFN-d)SM%V^+nRkl-=p%@X(;Er3(PT(JN?W24{hnn<}VX*bq{rA3a zX9b_}E9Vs?%?B^M?oD&9?-#os0G33HJQB>V-x;5`F~Vaf^OGfg87Pu@P{x z*JQf;7QNX+*+3%Fa2rs!V? zCsydcEwE2b{KJaSfX*ZfKYt@fn{p#hQBPFKQ`B6&&2P_(D>&9jC&I%$P^EYK?Q@)^ z$LY`f4!Vb?r%| zWy?b?l%eV8uXLR$Gat8nELR}-M8fM(Z9A^|VCfq-80ilpZ45~(gDFXS-wkG7EL&pF zjvG8U?T%2|YnPCA{9x&^A?b4>^`2;IH4x7E!1a+QoZvHzT-9c}sVNjug}dy->tz_N z@D-e7au^Kfu=o1?_x`|ey&=h7>hzwfA5W6|&uRK^!t~R$eFOuc_B4Ik5%{gs^|eRP zUCWXW8W3%nCfmrC;_8aXmSUFnE|s0t$tFs5c*3l&&_^z10#~j-s8I_l_#pz;v(Ba5 zN5@XpA1t-ccjo+AAAh8MktgLe6Fb8wm0vs3KGu_RdMF3|qMQ5qvSfRKqgLOz%-*CM zk{JQs9{otNeNN&>?3RLRVY#Y){#vt5!m4nc?z=_@HrYw~^kw!6XMIj^s_5Qj?0WyR z10@Zc>~hGI*OiPapAbziyO^bq&D+M{fK5pA=PiiTJ| z)~@o9h37TTl-ctN2$GHHcqrO%INF+WQ_Uln+p9e3ubF#broQnz)U{R1?RaO_Yx;rZ z_DWCI|H{Th=FQd9Qsm<1>f?pAxq7E?C|5s_VxQqj$}^S96D3K!{?9(oR6YM_dzLdT zPp>@M-n{6X#86ihQa?X`y`dBx;U)OdD={lf&=;>9&7HjeRa3zwejkSZ2a`oEcm6ojjh2pXaL6 zoV8n}7Z&S`m247ww(5;5xgJ%8`i7PEiLR7wEMs1D(KNcMJ9hxBkA9xBzlj$@;i@`@CHj2=0TNm2HSlzpNe^z-aKe!-+NMvEqu zF(a8$>-lTB(Bzt}pH8AZ!)cU1?{M2QPdeE@DMDlvOxdLutg^3hjTDdT1-9rISJAwb z%k=B3xbkV2>FCw=Ig4(b7plHoMahTg!R5X&JeuCPEW8m|ZJ*}J*lEN?`o-f=Q_Wdp zFL&nb)TgbnZ*mU4tzTbb-RlZ!X2Hdio@U2NjZ_v9`B?jkt>5ezVl$=8$0w@lB%N2WG5RG z`B*pakcX#Ly7ELGo1J#}xJW|ZqaeW=+SC)sfx z_f;n6c751hd%4a&iLqY)wm$15d!wuJo&V?uPh*eQd6J#sjD1%-Gws#Rx-0eZnT*K# zEA>^vs(1C1{FKq6ue}F#cFvV1YhUzd@Aud~+-TPr+J zzS+88cYcqK_z$Ps3lkmT62c35MJS_@aK-j+6-CipeP}ih1^!!1F+*Xg4oNS}woh>N z+!h*jJ=wP9DZAao&D4kW*oW!;+4g*ALXDokfi))cO}$Y#P@`|yz-@c*XC`j4-n`Ef zrw2FK3lcwy2{k^vux8$BB1OjKhvx;yrOw&NQ*lbDxX9Rh{ye>FBh$YlB#4ZjT?j;X zGO>}l@n-ZvrL@+VQpUrVc{b6H<1%mD#FJb>sG?or4*0x4uG=X2J00 zaPyhit>@=3#rZX1Xw=y-lDgj!CG4r5=2Jm zjxGABGw8O0T9Z|0IK>HDBK!YI=I_}(^h8%EZY1{@a?d6AoINHJU%0#8J4Zj2EA_d{ z2qKf|?sN3?JbA>v+X(#O6uIY!1$%{WBqT@;3)Y+y8pyBZ*-L!AbtYYYSR4};A9|*} zBra7Oagy6K$9g+oPP`nhuRRm>u=hSAm1Rmk)>rvrZR8f!U<7<kaeAdf%Be%=eHH zG>20xq5jvh<2?T?d$F&iDO6fmI^~@G_x{Kp^>=5nc4t0pr0HSlhxtYteKrr636JPI z3q^@HhGj3~5xw(l#!&tvW(;LgIaz^Ap41cr&f z_s?Sl);?*YLcNc7&ztDO>IZKig{0^Hu!8NQ=XBV@WPgVzLGK z=T`OtRWF6I4qxm`Lj|ucw0YImt8z_`@uZKNSU&yH0(RK{Dzq0TzQY>#Kab86T_zH1 zqG^LZLoeTEpK3S#$q162`m+Ka6n?plsb2XP6If{?qbond45qKQp%Uo~Nh2%L*=Lyk zSiRjoDz2>Cq-_nSU2ujd{#VVGE=Yda2<3hxAM2~l`oY~ObSDs|hJ-3y$LBZe4|j7E zBIg1W5`7_QWUAe~Ir0F%JTB!GlW`>N37gGQ`|$<#(zyPRG%`8nZ59R4s@X;InDDAe z8d-Ti$HqYFZ;~3D@b_9KGN!?tH=XQ*M^Rz8({!QQE`DdV=JU7<}^?!31NOnSZz`>Z*%kswUj=Fur?%24wpVJM@H_^dh`z4nw^R5qvUmycjQ{Qh>=U5 z6>dkn+(Zoj8zx3%;_clOQTr~?gJnb;BU)i7T2Z(jyEo}KcG6G|z;Gsx0a_M{5xI7! zZ!*gGDufjWC!ZZkE=TI(`-o8+ijf^_O0eGE7{#ru$bmEqer8mk#x=7#IV8`m_OYH<# z-H*XtVDMXSsZ+n#yYmqzl(5%&`qP7Q%l^jHN>to%A_Hs61{qk(;wnOd$e4UD zTgK$txY~m=y*pdX9&6(|LW0h4T{dOwT_Q-{Z>kZw2Q10fdqvP55=8dAG1+>0Dea9t zU{XYGC(oa5R+O7cSy5U-(nzx1r|Z3v%&Wc+szXE*CLimh)6L%Zn4gfWHze&fLS-#L z`V*>=<1oqOFot}rf1alIvcvX`xs0GZEO`DjtV}9|Tw{`<065L8)Yo6m zZNDR0KYlqIqOoZG!R6fC2R(Y)6)3VYWA(->*!v_-HL07;t>gDsu=gpLuHU_a-9fci zPb-7vUVTa#d!KVY3XQyMit_VU8)Xt#g^LbknRT<2fUb}*a_Kf9+{v=miLn1*;gq0o zg|57kg?=O?m6-Ce{<8l6+%?P4%?mX@GWK?_H)GGfim?}OnHpw=uf^8&=0+8LD<@A! zb2%}!=~+8!wmjGkPBC#KSLV3&`tGY3gk{r=AhPrG3m7oZx}#JOXM6Sg-+0Y!#k5 z<3Ur`&2f=7LM8}#&^2YgoN1n_H(rD7R>ggY(M!QY%@ZVe;kH0Cv3R4Cc< zvG$%6w9RSnbr=i=67*5m*(Z9k{U%nCKKd|Js}Ejh&vAN>(a&9HpW@0p=6ii$CVHTR z3VV6tJ%0^VJ)g|{{PoW;Nh38p?!@5UCa!X(5wsFXK9=jm;2DH_ujk4QhJ=c43ku&} zXLcKfBCU)wNh7;2|uQNmRyX%?cRX&q8vg94RPFAN?vr`#o z6Cq(_)_j#=W{ns%QVuaGBfE{g8IyJ!eZ;7lWnwg%N~8EBd1SYd$en8-6r(d7J-QH9hJ@AOJeFk0rX*yu_PIH)7;lo-hkI||33_^sed_Eo zgxPb95QSXuW4&~OnU#O5WLCz`HPXmcxtpZ1bPsQlP`w8WbA!Sa^JESc5g}!sNgWxd zaVMCrM}@RBt|lbS4Bx_EJ>HyqTzV5*)6)4SZ9E+*AM2mTn|j@J6ZJ|9jljr)|K9Pk zl+M;~iF7a|O$oQ@(c?|Ii*Kgf_QMXU)vi#^*WOIpgoQ?05Kg;1Bz;AsNrxM$6|VCX zq(OC2m3?Gf{~{xeJd__9D)*`?d#SH~v5^jhbN+E{Xv^|e6{^zsrFz;=cZKL4jo zq={sf6TY9Jdwz-)p}#&Q#h&ZQKgNVL z>cgI9q8{~$XNFxwXAV;ZYlZ&c7OZ~x$C)^#+VK{O-V1NFXFF?;(>L78gXIygho)~P ziTwOEC50k~+wj$D9nd`UeRnIzScaDxX*n_FW4*t6(n$?j_dAX>F(SSD=xTFS@BEcL zE%;<9SJWM9UgTcz&}wtu`Sof_X$mzoKRhz-UafzuwtwP`U#WNA#+@@^rG7xzyi&h@ z8);7JQyIX%c6%`;YU7`ZyA0Z`%KHn~a`CuU3bmwuXE7)-=8M4t5>GX{P+h zEWR#HX7Pr&_%%k*74DDqX>v{@_Dk&)z z78xmd3yl($yd)_V7MZoltgJ|}&`E}hMP)`wNq?zKG0DsF@qWYn##p~G zx3b)PeeXHvd(U&u^G%=ah%2Xb`0qzHA%PvN#S5`qC&E1EOSN+5bJA1~*hp z)RcvkVU#djGub=+c!SzKAVwH2m;VUeE5DGcs!|v(Upb&3`$~EQlYl%Dkp(^O?8h(u z+!WvtMyj^zE4tC18PRtkg{!Zx?aO_gUOg0st0Ou4Dkn2CgyE{60eyKkxhbGf7_OPb z-j`>Rn*w}BB9CgF`ISCu<||Z=+)-fUYjcnE;hBXxi7?7EZ8XRfD&DY)Z7D~oGb;4rusU?ZXEy-JQn-r;5ET)rw<6bb5HDhKZ z9grTU9boe%v;*{?0Jfjjx>jplLWg+;lfa#){Wx(KE+JFPCxa2CtNfctxE@bMuBUVg zxTTuAE|RA#YV>Jf)WAqOScqi5*e4+90Wd;z3M%?n$~M+FAr*{ys@(>1&BJqTExmTn zyr3F4ws%kqa>+$?|J>rDB1M;Lo%4KewU1tSFki!qrJUJK6c?BsXkpwL;VN3z1KY8;`VpYXZwQ?PSS02sF&*by6wimlr%70!|#a~50ap^KrTqxY_I_dKfE|VKA zS#AppNKKcz&{>ZZ37!#E@^Vr#e-=b4wUXfx>WSqv;G!P|V~$SSJAyZ&A?l~)ww3|& z9s@U4a|cCWciUzKcjd>yJ*}1YjKGw1^*L~U7%e=K=F-9=fW9|Sy29p7zbJn}?F*#0 zGxmQhZJI#o9rVd~0{UFl@9c?Dd{aIhQs!B?{d$^*da5X&o=KMpcb>Mzt0zBCxpB!# zVxAYKpDxC(9#>DzR5FS97R*83^VLJk=w1DnS5l69g_f;#PVeD5O6wdDBHh)s>K)KS z4S&lPp~nAATbJZFDBtZO;##VBQNH{dI5kZ3fU2jEWr|jRr8|z`j=n)H7V$K=u1@h; zcYZagSBrS^H`!L^LX+7ipxVlqB&qgNs)V^vS*kPa*PRD{pMb5xsL+fS-O-RY7f>@N zK7(W~FMcju%`Bj^ic(>0)ycjIr`71sws7+qL5(-P5Az(7x>jNDg{vzCJgHffE zJ{HcE7Z-Xvbru*PrGWEJ)T7_Qb?+l?Ro zbq_c%4A%^^rkh&xHo5Ki7m>o%+LOAuo>+7a%o46^FVVZ(KPurAEn6RXhaM)!XCZN{ zc1Nv9>pN7?#06mZ=~*J6o4WE2)wyF47_O@6O$O5UPSvDUH1Hg+fa}@@wCJW5t|BW_ zUIpW*uDDZO)uC1NG&MFy4Sm6zHF)%4Pi_v`Ar)pTZ*zYM%e)ufOH#!c_pdV7Q_ z-$L6h^^?hl`ERIC-=i%@b%C-K+HUn6^$z1(Rc@gz#J9kS<=j-f0P;hjEx@zFss1XY zJfq)*-g2y{Z+)NM-o2kXyV>|jq}f5Ad=aYDwKs{53f{L(@XC3goQSXO#`^5$9UrKh zKcFHXpfi|9XepWV0ZlXUYxz7YSZ)8n7U&r(poV}~^jexC>8pX}#KRR_u@HH;vhC98nQeRCh zqPKIsOVx`-wlQAkODUnv-V*QDo98o)KD609D>td&4{2`}`o#tH#93c@)$l`ZiOp*1 zhjdbxyjkrbu5aFEm^4Z~-O3)IqCXh#+zw)p0KcW;VWoq?DbY8cLHjrdyaSxaq|woP>{=4Ri9Oy;2p{`#3tXkEpm zU&AuWx2cbcshC0A&73={;E$=A3o_Kuk16NM?P}aQZb_;X`l%WtO^sb=>(|7rBOjuD z2kqO{)92`0_g(&DYvG$!n>LMUK|3H9soFX0Vd}&G*xLHm2u;#nV7d9_kPA_Rx6#WH z{nydU5lK7Qgn^!!yVTNkwp)h=RzimQ5khPuhi85xBZN>8JuS!+e$G|Czh_n>^Q+xa z#NWCf=}#M~(PiIGDxt*Yutoj_Vq85C^O+ca9K;wQRx>f=G>GSg81o4c%PxRu2r-+9 z>HmOuz#BR%ndswn(OiCL$%s#h7}*EJRURzaSv~!!t)FLdKlSOS+>%ks_8HYf!i}oy zXSNxhi38PAj?WKPw)Ipm^M|OR6nkafdeQY>Vk^(AVQSlY+ZbC|0#wI%st=k{)sOg` z`mg^8HT!eh7~hISFlpb_W zU+qsXm?fz_8_15>dsXm8ip%a*<0$ruyYC`SN6R--zg&Nx+QZ!F@hZ5KxTWKTn_WuW z$YiyYsU^um_1{F)@Chn-6QwDdpvF<`6*1A|-dMkye)u8db+u;`_0ZIbs_7T>zDU$m z^}rW|)6>)&U)YA(!X81{(i^J{c6$reWHT)_Q|79!n@MBx^Xh@kV`7nCv8vsQkYqyTZOz!Cz7ftlyzt{F0?B)V43l zggL*c_FD+Y{iY^vu}${edq8d5LTyle2&T+yq#D(uCDO>RY#lvA&Z^m8(YZ>+IWP<4 z1-h+dX6$(typYts_6qsuBA*bOhr&Ldq5r6DCijZ6eN7G^+Vhec@U^YGXO6de z;A=`&-bk(f+IFigvhgLeuO)b@1vS2HROVq{ZxynQ9A|VB)qfkgppX`jn5XuCXzQjj z={Flx%r`dgh7~lLl7*SBP6XLQ)v;~1R$e(lm(a<&e?uqFyMxrwZ>XD81z)AMs9#0j z8O{$;PkB>!yZ8

sW8cstdyUwbuiJJoVVs9L(6x?fahy{x(5Nwf$RLpe?col7)C;T={=zb9ff?Qa5}@jw`R1nnF*S zsEg%Pl=9ou4LixM_*bDmTY7bE#|3IDM=o9MuA8Z4COXq;MT}^09wbu7XTd*x*FKDC0 z(eXR>D)&cv2-=5!G}K%c#FN{0&?iool?i%HIR3KQ@guFa(quW9sTmzEtF}K8qefPJ z1>*R9{-{wif1)*PnXKz**=Yu!_i8SyqS3T)cy5Qy^~H0}DCZ@(>@+R;OpUto6RjwZ z3gdugOsY|XDk#%TSrm@ajNUbBVFjhg5k|RYT)CunRM4u``(>FHG~=^Ns^`zdm@f=o z?r`vV?2=meGpWiKhHG~?l2rXn1q+phr)x9S`4Sb(j|R$#iq~7yzjINo*+Zo|AdE6tV)#tDsIKfG8{-zj_|uxv_M#g93o-Tz&_k`gRH}SoWa<>RUAU|qGi_0-a<9!hFjbb#t~P2x z?1)9gj`)?>5jn^&U8ml4{<4b6CUz#V1M-EvTeBC^7nLgcE2zKPS_US|0@`KMBubp$ z*cQ_wn-)JumqNA6uomalpx>yvB3=i>)j)sHcfNn4q7(}wOXs%n+~p-BC^vd9z~B6V zBjvx3j4sK80@qgQiF4GD)${{61NYgQ2c|9uyF#m(PVAUQ)Pd*j<0sJhU?*#KhjVJ< zKC-KNB^ariv5$WItCAQ+!{CLY{lM{g{j3^aNn4N8>*ZfB*3?%z-6Yn3Hn$LO(q z>WpgqJAHS*Pnpd56gy9Z>X{Mcuj6YRw;%ihjG1E`YZ% z-!z${1Cl?tFov6qJI&We%PD2vuMmjj6)y#!OQ)3c098ubpJ3204T7=el-kD($1yPC z^*Q*YQ)K zx1+sJs*MM!4GY^?7*|d(V{i+)M%kF-0Mk{m-<(i04^gW5?Z9wtexE;~b{wWJVjVw3 zXXsPAfE%tK1>AB%b*`eXc>44N!{z20oRFToyoz4=oGUo)Tvcr^Fj6ap2Snu%}+ju9;6;+Tf$M z8v??90edTQrUS_1YWz`hm#2l{I_&6qT&*}thYaiYB8BT1;nLsASwq|S2FGlz1M`0a zH%e#s)!&!ZoPECS)wpA}{(}$;UYqxl6EsOwUx0=%UAl8e)c6y0E>}d}ul%Bg`IE`)6Hs)?!dPcA zx|@#%V_MNWBYs%@6lk^Th&q0PykvDlFkI(f{b$0q1DdhxFbG$_{%=07bli4x2Kp3Uk*tU{2eQolLEr5SeD3U(@^xPx7 zEp)hQzQJL&@eB>+Tn8BQbbTDE;)$@cYIByVBd!CuK{|)es?_kaWN=U@7)9FP=d0Al z3)BJMKT9vuj_L@mtG({7Qdju9B}dQNe1eNQflGIvnZaj36>2P5-PMhD99_=Y>;dy5 zAmn;1HK2;8oG>->99f+$8<{v?QE};zdgmNDu8e3T{l_6`tf$|jXZo|w)0@$&21Ca6By-&%6z%i#wibbfsg~wmcj&T8j|CH8j(uB^K zr(LFglqDOqY+XHn9OVApNxgrW+PdsXs7}{~c=MpzcbWQk^;2NvcNWz%5ArIZBmXpR zL_p0u;D+hd<1>fU!hdWNe1l5Bn2PhHuJ_Y8>u{0)#is;hIw0oB;key3;97ImMeoqjlMyZV6R;_uWyo|FZ}J-jGR zn5ypcvJdo3|4uFUBI`2ExQ8v2lx`(uS>LHf-u6x`-`CrIt7pm%K3%!hvtozZ;%&dN zwuiCVCQP~m3>Mzun!6F{uT^}W~|$%jyJIP z3D_%)JZ+|PpX%pB1&jI)DW>WackEL;n$xe?zT!g-leI&A;bXtSYyS6_%tsL_!q@KY zwVsgqZR#XnyN_?mE@((r6WY_GfaiSe%{{YrspY=Z&{@0H-@eqR;(t($Y@~1M4{D%| z8ZNU!J!iAiq2(XyH=BL3um2(F@aBgz&D2D@J=im_NM&Vsy8%CqE@8pd(eX|bpa}jn&INYz{iY-k)KHD#IJA*^yD6Dc<*%9tP&JkZ(~maMc&Q3j zO9H5R!&|8>0rnA|6(K4(kkb1ks&Rolp6K_;``G#}`I=5P%c%$KZe3#@j7H#ll}7{Z ze&(;S7}N?EUM{r@wc1P}`c>Zx3x4BjYZflsf2B z*b0eV)6^e3d0E_6jcIOg6POZsrE9b((e#aU-h4+f*sTfWz*CH4#K;qsnv`RJfJRe zoSLeJwxhnBm8#~pqrtO%rrOhv9PNS*9JIujSGuA!8NcdmLM zl*U=uYicRS8Lz2r9B+M1UF0}usp{I1;;Bp3IF3t}suwx-d0l%;q zxD69dH2uxN=bO@X>pI(S4_a5*+3slWtmrOBDqtIF-{Z9A5y2xUrsD(SfY7H4y&)}` zTDqavCpsU3S>^ei4L9;m<2GS@XUB3qP%`*kFzpP@U{yGP((wwwN*_#ghZ}kyqTh2J zozB&Ji2+tc;X;q`gJH7RGo}dEhFa4#qjbEkveE_Sr|)88CQh|X1SrL%r6xs|?!=uT#QD2gW1-y9l|i{&y~>GVS=MpHNRr%4gZ2kwSJ zljv^_b7%$Zj{Y3cSr5r``3*xSXie$eB?u!rm&}?$PolSTLysbQm>c?FqTf)9j{J=& zl&iapi|Z)C7gtL_>aG^x8=`lrm4V4NI*Fbqr0>u(Kr`evX~@~qqTKwSe#pj8y?Beg zlQVH6)hUG{Nx(ygL#{>sYvEJNn88{@>j+1Qh$$;tmY-TI;b zeGU*gU-f@HOlk4Ci};3`Q_mjm1+eDVTju=*eE01t~? zT@KKqr)~#mZ;a8vK7&IK!Fn`U4fsC}5cOo#u>y2Czz7L%W;*|2x2Yr0>+_05d%NYdgSfNl=#qd{_AY<^Xq!TwM;(qNi>L zcuEv$2iUv`Mu#2^|6g~2o~1vi-~?JRR{xGVt!u?u)dVZX{y$>Ql;7eo7E_nz4PdoY zOj~h)jhmvPbPMzn>=3?H&u>KMv{rgmZMxIY8DAs@n_^AiMde`nbN`az$0W!7(wcrf z(YZiY`bMIM@_m{;15h&EspAD>#eIFk#>8;AQOGw?xicV*=)B{#8o>01ykfI5CJ-Z{ z4mw}!WR-78^4y=T^sYp2=Z4;m=sktb`Ma862r;_4NidY?{1DKZLA+$JKx&M4IUL@y z;Io38fqktWGk?R^%P_hV)0#mf(cQIOU!wEzi9HZarzhD?g0;IDB$!Lf!K%Nh7v;}; zb1S0_WzfV8J%s4&DzasCESLBriAyEkA#wg))XOLgkx@3=M25lpKHkd@ak^n}Ra+CR zqXe9>m436-gq1#o=zZMC4<$M`la_C(E#d^+SwfnjN=K_+$@aF+GI1lNVrK@4a7d&N z3!bJcyEufI|6`=`pRpOB*TpY=416tDG0JXgC=vS9874o_h`=94jKs+^Jk)?xdhW8p zL%lcI-p2Vj6dm1wIAJstRec7WE$Or#O(#8OhGCNt$C>DA0S-t3>f&z|zEcZsm5jFw zqd+onNE|0|io`LJadI6Q*TvTvOTJDcW(-QKGoJG=8OJ_?5*I#=NL<4Ls^JiOt1gkJAd)j3F*|e)I`)2+TnPuY{|Vm~ zUd8u&sLv)* z*-p0!mPvvuVu+r0TU5Xhedu_E6& zVT)jgi45bQMS<1O7BsoBNljbAFkU-wm?T(dY)$8Uygu4&p(Ww(f^`Y43HMMsE}fPB z3(>hr{DNT!FY`E9<@S;sN!)Bii(nY!j7J4`5^POaA-y+G^Yl{}u z`=wW<YiG8VXzkq4eQ`e=J=XUg4ZSt6K%F_Z0P9P%J=#a)OCB-ZrZ zg0m!cn5nM%>8MPktBb$YY4Up;TChYi-YyD{N~|+3LIPtqz1K5^_GY;cn(w~K zPYR{-$ne06`l7G2&Dq-Ujf1XEu%!zzuU9O&EE44%QJ`-H@0IwZ#9kf1 zZz^%6$cGBxi!NUXma5n;BfKtt5%^9+3&y3Q2*r{x zMl$f1xL90EwBRhsIIfP2>*7nsk}q|ZQcK38{w3q|e<^WY{Ms2)XL*nc%o`b9;_QFP zI8~hDR+00OPEbA?O)*Mv-bmmokqeY`6@qIdK236WaqZ392P7AB7A8 z-DPO;&PCF{#5+f2P*S+$Sgyx9z4MncB(2_g(0&Xyy++O^`*|tm-x+)xW?e;^q=SJ3gl zVT|qSi&KTRsB9xc1ayK%f1u~23ZH+Zmw8VJUAN-zG@tXT?L0p|reN(janFG6KNoS8 z$Q}M8&VAPmf6l$4?B_vG6uwq;d7tX~h`onLtXespPU@2uL98GPaq7#6?>T_%?lrBX zvl1##Bl&Q?`w79kLgw&^q`L>u=u0uznU&5Xk@gc_MzcTFMY5+(Lv%)>&&ZJs zVJ5AU{;XZ`d{oSv5i9*Uq1TAKHZYfDsYp)4%2+`RE|rzO1_`dCFBW>W7|2F&u%`c$ zXgr1+grg>JVj*_*>sO#mdg+LA(K(p>zGeC{@S~R_jui#}wjp0Ca{to}Ik&CekS8AW zQFpv(r{6W1W%u`*n(wJzT4HbGu}qCip|hDhT3NqpZ*x~3toZpL>`XF?L8o$5&+Kxo z^dUs&&S0f~?~RW2^Fj5>qtp-LrF8i#p?_+3RFk&8?;+$jLFY$t7=5hmeFxEa!)m3o z=jP*VD}5Nzs9BuXG3c&(gU^$u_PDma$n}dMQ;5f5GK~;EUrYZ0hFhD_u|}G$5V7C; zh~E;uXh!M>mV**QAFNywL0uENgIVQuGdg{uMppXS!pIaShRlp$?!FWAx* zrWv#TLOW^~-Ig|kEs3{F=z4^#BpQ!AEB$q%vBs!v&~NFmPNQHJb?LeVT>SZ>z&e+u z{x7C?e`HEMvCk2daI$HJVa%hLEzT3VUjBVfG&)(kj=qj)L+SI9rRJ>Zza$y1xsOGz zBmHfT1gqdzM0e0zd2at-Xad zA@7KqJ%_wx#a6Ug(B9f{^P{LSy$ng9G!FVaBlL;%HwQL|UH*L0&;5hLL6MsSXda01 z;d4lpv}$%p|4kIS-35?U9LL zm77jD%8PvcW!PbPH9;kjRoo;e8n`MjC&{WQqLI!=<^IGX_=v6w67@c4}F}NMTx*QeWBhrU0E&Wj{7AD%n30rROOS1r3r|ELc7+B)c4>Y#t; zhR*pLdx_z$st-i$NDJ_S}RgW4n&QnbpTG6xas;( z5CS_PPVA)iJWcO|BufVU&E!-I$Dv?~;F)9~*M^nOue^03`cg^Hwa39K*GmdAOZ2j% z;^3eUd0D5iUlGtGtbxxOUAUf1@)oxGk*- z#t^+n9dsTQR{1U@@2&t7h|UhoDu0eUojXfgP$zQN1h)~rl^aDL65U+}-_=3iO>{1> zb51`V42GdM9vM8&C|DOL@kyX(+=V!11maI5J-cWQolao5=^d2bp)C^n42T)W>dW}I zGERVDMBR-H5+niC($`icm%*hC?^{uwSv#ClQTDQ+F}wW%`@LuachEoE&=7hx5Pl6e>nf30;WR@*28R zGkB!%sIVqrL-<74O7BE8E_}K|fvO}_ z)88Cc(T4}!{h~XPi4?4I=ZVIxWTo>?AnYpLX=ug#2IY-c87?|cuq%T=Zao3zZ;&gX|F2J| zUuL%|)GxDJ>H1}MD_y_KZl$v?c80picoZ>s3;DOSvF_NPN?*ueYbhkthu0=qXHbCN z=h0X6h}M0`?m2Cqp`>}43mBX{m*`>qbS&9QVnCJB_meMhFIx54{|}Q z^wvaU=eqK|d_`7U+>VGn>C^ZEN}_!Tf0>0dvC;#H&OWE%K$I|;{^szKq}MLsC!(t&5oF<9_o#9fEn;;5#D2CA|Q{=SMdQUL2r;{T|mx^>wgAe~X zgYurVs-du3;tGlPO8kQ;;`ZcVeTz0U23cr_WPKHMAmzjJSfa>Br2>g>%qgWWc_HLz&>Z4EkI6+=)3%t3k~^1ZdnxaXs$;j+>!aE4oi`UlGg^%p-xr z9>F}tbFeDr8J%mi;Vsaso8nHv{WuYYd4$dJ$333IFBS{+8<%%phUJM6Gn~{XxWifL z^N7wq-%5YB4*CKQ%AbdiHNiqk;EukS=+SQEd4}cMv+8+;=-e@_bbfaohI;s@mloN( z_YNO|-WMrxl*DgK(MOOL4jgkY8vBorIzkDZVYedTihoc?!&!>LpVR{E_1)3ixuO3| z@=QN{2O6yr{mtRH=)Dur;BDN^^!H6y%>10@SC5#dU*uW>h7l(TzNVOcW2<;5#{Y(~ zPvpjn+)#?!(#J}VB^oz&L^2BMpuahwxSgq#bGWn7Z(`mD0Wh=xb?>i;kHto;RMsFY zPs|B~ldhyfaqqU$r_j*jcFh<0dsvtPKVjl>IHgQ_fn$}sm2&1X3EgS_bPg*>76sM> zLmOb|Sm|F;(K=jJH1gnmH=z<5f_f!{NXpx8>^LNw|OMvXKTcP9?xr6z9k z5ND&egxa~#NHPV(%>U`6o-E}Bhn2p7=sdkz>C1@D1IsXs% z()-sz=luiEdeMOw|T2p^u4aT#MvB{ z6ur!J^l*b>w$w_0nP^?y(DOvTm8ZJ6gx<(`FaZ{DrW~v|`%a8lUM6wSBR6O#6^%mP z?chh>A+d~b>xL-o82Hsk5QjtFY1EZLwPX;=Cn6L=B!jx-bIV~+dK~2AMc(RMrx81p z^q~_{bL(EHe^@Ba6E(s;l>;xQcynQ;#}JJbS?TO(+Pa}nA=-6%xVufJPdhgWcwxx% zh&6*WqH!;`(s|L)o<3=o0(?R=?jlzDDr$2l4+|@UzbL}(XQl5X8c&H4NAOZn~;x5Tl)&1bjx#>|-614;#_!|`R&lPfjU4Qm@( zxe|jdy~<#5&^u6UrGHB_URqh{{7t(EH}ocT&=+y}-DOZfbZ$#)25;3tf2j^V+?2c* zu<9uzI`43t=`#McYb}$iaS)S36n%I+^4Q>n8G zZr}yXBZR^|#z?WS)d(>3{Qaoleh{P+T=3ay4(n+g^Xz7&^CKoM%up|N@^yMzoElGp z6w*E+28C47!|s+*;`oWTJ!k%aPt`&e@#6DtH44aua+h8F5GE@PcHd`xLh|%URRwjpufe zmnt)M(fXks$%9TqL^CcDgZE_C1T{qO_|vC8uh%_IhIk*taq z6P zeL$UptF+JEp!15E1G|3?{tsZ3I81c4muJW}LIzek$5U5KN9MBHU8s&JUtHdj`~+T*mQLh%kMQ{*5`!%?tQRY7H48I%Z6dNb7JoaPLHbMnfTKb&FpVfxm3i2VhJNL*L9N)`D6FQi-VjW{C})mv$<903` z0z+Svb?)mDmkYg8;`fDKD)ILc@00i##pdYn90i4qByK5j4~cJ)c!b2s5KXw@SQA;=>Z3 zmDoeiQDdA?P%0TXMBz@EGk%x&gv7=@kZ&q+2Z?)193knW1m7a@of40isvjqKic|i0 zMB;f8FOj%F;!h>sBJox+c&Fe!68|Z2wZxYt_LVuOnZ)Ii-q~3QeIy<%ae~B?B%UGh z;}Sn7@gj*|mpEVI^OC`B)L}WCklxR(o5QQ{BTl20Y+v=H#9v_GwVjpDFQ2j7=A%gt zT9H1Pv>s*Fb%^vfrDZpb(`)Mg$pkp8B#U^Q=9_K@Kitmr#k4r%EUk2XtZk)#SckmZ z(`p`n|5uNz>nichKByt>!vCf`*l$?r?8=9+sFjgV40b?PdI8bd^I7RjiO$V;vyYni zzP-oDlrOO!dr%C~O|^<-crvmc1pY5PeNlhBPe-sRU)R@bcdyvs2Cl#7e_0>D)ROTv z<;;u25KkiUSMQw%f*Mekw6EF%qbek}8lg`kt@Q7xaJ=x@Acf&B!httLJP%pvJbhe4 zcba{QXy%t6u3@<7lZnS`PphJtMB`c6N>3vi&r4SN!$fO$9i2KDGvcme;0H5;-O$sC z#>;(c22T;qUB<;k@8m}QW1_X=4sKQSBGGPlLw|{AZR?=(BG{^D8PWQ&%YAxR(CB(2rS04zEFYDQ~5-G4ANyuBV&%o2%nSZWN6pT9_L;*ATBstQohV z&d3WSE4>%ddBj@jksg#kyWmEX;S+A8zd0041@;CsZWF#Yhn_@nBJ(~rGWUG>4vnqc_Za*3_yG~OM+pCfUp$n~6t>d`%IrnHkoszcB3*+S|CM-!d>@;8!!HvC2EX*^~R2%Wn-2feVk zB6N0@bbL<_07(&h3^ffU(kFWF31(|KED=oNW@u@Hd^^EB!!Tul(DnZRQKIqmX{A5L zbRK6`Mn)Y8o~VQV1<~DQe2M7njja0l`$J8IZjN(p>1-mpv-I&g==|28J9&OB-yOZ- z^>p}|hl#<9JFBIS5uJ^*()k@2T>@g8G92^Io4PA8KcZ+)e5;}TCs+K3V`eY&)z zUOhZXv}SJT?-I?O{OUUB?`gWzlEDX>VWF?7gU(->;2~%=l&hNigq0p+p*wZS-mfEr z!aC^uO(}OJ{<%qc@FLcO~dcbay4-A0u){|AXkfoUxYpcpdcZCY{=UsZ@0b zO3dLk`ZS`CmA;N>Tw*J|qz?Kgb z;o1(fLy$A!kWNJQS?=h(mbB9OWg_k!R{G*P=u1pG_2)RrSU3N3q=dWJ7qejy^?Nf| zc!lwe5p_W3T zSoPg5^1O29aIHM&Z%mOI=*WNqw_9KOSn1THW{F=Gd0u{Sc#n$0rAc`jCC5^Mei=~k zjiwacA17E3;NPB>gR%A^P;V|2aJzG8Az1TcELCac-zf6Bq{9XC9LFI^u&&v;bLFXw zUM*kaGRy>B#Dxr(I91|TBrcKIQwEi{#Pu_L*X`_VAMN2hKM~F5JqfYD#32%gNgOG0 zjKuL0CrX?mahk+)B%UvEj>P#A7b2!_NeiJ&;&O@iN?axJX^D*#80sf+ki?-9M@Sqk zah$}XBuu z9VLWVi4!DFl6b1b=@Ms3oF#Ft#03%;NxWX-trG8+xKiSy6r1Dgyb!!6OCOLpMB*@s zBPEWJI9}pJiBlv_lX#BA^CixaIA7wz$v*56Y2XQ=Ok(_sK*U_%{vHms{|kEyk6c9l zs>x;vR&7wTH`|A`OqL(UFcq@S2#nQCwSTj{vq!=@-!gknv?D0ZKhw)6 z)7wAIC)11mqsY≻Ww#H~aOGXRpS(^WJ`6HW;}52XDjMDEgiMUscNgm%mhx{x9A_ zF4Xh>ho?^fvP7D~R4k8SQwzjng~TB|pE=#_RuJwEcNbH}4;B z)uW+bc0$B?kB9>v`JNS?kztN3kG&o-K|!erA%z}Ap7T6Td&cJ0Tjg2oG0&rfWJ*0U zJre!n!%97Ky$c&>)z9{>@Zx_5{IWc@2551n1+N?fex@I)@xEti?L_>e;-O~KGLdnTOeV+a z%g&5*1V2Mx&thB(OuwH)yD`onwlNg0tO3rGwMFdxo+dxa0e(68UY6h64)`1L-K;QQ zeDZ`tFvwgQlE`b5kzPOB%%+P>Sl%a@(^K#MB@zl^CosVSsCX^_*8UngRCI0~H5}m#RTu3hsat*}D%M9aJAic(uX!>M+n((u}f&5et zB2pM}Q1bKO?_5xrJOKi8c&ALkuLlJ3ynqurqZ6I>20qybohXdjmcxB_04E53fqq7o zJ6W3Grz+7YYXrZ28*qh__XZSxr&AJ65X4&o3ge?75Fyw&034_JOMr_7M>>GLeULtK zI1F$Ier6Q#cEKqpk)N~17YXiN3eBg4nnnLx6o#Qm>&EC_m-6;sJL=wWqK@XPH{ zBWd@5|MP0eK=5<7pwnjYnJe5Jx;Vw;pST+ePYb^DdGPgZHqDzNy}o2d*HAv6+EDoVS*m}|Fn1aVy2?Tx z0QMOMh5AC5m>XeW?sV{VjYUx3;h<}5s^FAcV8Btq_K%Q$`-710+>Yv>+f-klGmA>f zSl{FG`*0W#GXn{pTo1m!X6My#G>N|F=TZ=`zJ90WGe|EN4H=uLGqZkudycnrJOuOw zLN}Scl8*f)!vyk(Ttg*-E2T-|`TcGRfeTTh(=<9bWYgDcSzceC6FC@IU&HfHAL!H9 z>^N&)1wr4}Gg}1oeLNvzfWE({`T!K-ULHDRI}5Du^BMgc^qJT4kU@*)fUocQnM9+3 z4U}tqoW{-c?K39m`+WM))`_vcDCqtEV)3Iyq_E)?RH44ZDPSyWP~Y|R`~dLvolnU- zz}Gh#HK9p}n*bLX(fD7|76g68QW7~N7Vvu<88)LfW`i;W=g??jTs#|mzjt9^!V|zf zX)DJ3{7m43;z@nx0x$eQ3iJYD%HJr?hr%>!OAZ6lAP}1keEeP+MvH(u(X02IK|Iw2 zhr$#XocJp6DyhL$f}IaRe*04J|M>(seHnjIkV2I#xhmg+!1Nhn067>AGpSQ@hCyq9 z-)jwAO2dgm_irGt@BN551MI`!(WYRdg$<`q6+B8hE!{0(=5I{63lgFQUq_N#VIQ^@9ZG?Fe(X`K;Mrr_A(4Mug4(%E*WO}ic0pN zoL*n_5ths8ssEX`&`^S5BcULbJRFB+^m|jBL8ag?dxKvkxcvzzG_MLF`Q-axAYGth zrhjMz`1)d$qhEus?;~08De<}gl^jHZS!a+y-=LxwjTOSzYrx_|kaw-|Xzj>Buj#f6 z)=RjA!{9H;hXFZ)(+5-gbCsk2r+^>14Namidds9?#0GB_ z{u}fpk8$={$UAABz}Sx*9EbP&!C-x7P3-Gb|16NmKi@{-lhaV3@uSVaFZTj}41IN$ zvelVJ79C&P6h>%i{_vgp%sE(XpS23=QkrL=tb!&!GphpLE#;V&-0&N z*HR{SFD4o|_cP%0 zHvx}Z5BbxBX#8`yHvxqCgCVdh6bUkJ22Q5paEbh4fw#7+ja$$&o@@CRo&wGm`7`1d zeE7G0DEP|kmp;S^LcKXikVvx<2i*ku;Yfn zQ}jXx8@MVS{HT4v#ZDS79KL)P2}*|nU%UYVVF|!v8v~EJ3;1j!;6%Z%3r-T;LvXU- zW^``L2BZkC%0Ux4rwZXMA*2b`vzdAAjagz0Lw%difKP$-r8*x)paA-2jft{o)wr`~ zZGH~9XfnuYXn{7#uu$V(ch$zl(Y5i)dr$%`-+o8!49>k>TmI&Awef0N?3(kxR#5&k z#=X9N=%qV=cT0^-nurqX`;cysGa!9UQ4o1dc2xShp`D2^P~Vu5Ncy-5G;Ys3C7%B^ z;k2W+K(BCMeN{)^DwIIuo>Im7W{usU;Om<=PSD{MXQ=Uy&(y{xbiiTm|Fyt48Mhh_ zp!u5>=sQ5V^sJ2+Q;l59Z$is(#(WnDhq&#)`ic(^85R1Dk;ut1|LfaK>d!y|jT_K8 z04vnEX+~|lB(*k9>QOuWJX+XXoBq5kaJ0TF%gFy+D{x_6ZG7a-+IU9G+6oub7K_vC zTUTOM*2WH6S6s^mrdj_=pTk9ea-B)!up45=-! zMO>}U;ETz%GrVC?Ed|C#T72@bJ3R)SZr5uVrutg5_g)6pH+yZSxq{`*tG#&sd$tP* zxinrm+&dNqC5#1rya_U_Acw=DRMvbNe@h39Tq8kA;NS8A3_dORU0NkG--m`PhdV9; zr!eOEZ~PC}5D1@GRT~$}BJ#W_n4f_R^6!JfuW2d98B`0tP=WNP$AdqS+MG)qNn1P) z?=DA;B@pKPjg$219t(ui!hl0ItyEZHjo{8xkf3rB`0@8ZVajCSo_WY{4y}$kY?5IZ zH5K@eP2h)21OD3pE}JIvf3Iv1e5OO7J*@-SAoCtFQdqwaO04gTI!p698>n$&dToA+ zcuviKxKs>y5c-?6)ABO^ACSpKCup-w3n0NDTIq0x`s%BX$AhnNnYeC^ACk?f#`=6A zTnq@Dj`Yz^AvEs~3-ujb7jF}bXTX3-M^WYZf*)T995oaCcRzuAmf)}6Lk9EHz}M5c zc~276=vbK@o%(XIg=tWzFPw^e9Br;|zMA5P8qim84H^x;|14yn*8}>pF+H0`2w%^p z`l_!-PMP^R4SfaIBX%g%S9PrpfI<2OEj|6}JH+(#t8Y2`iKc1pGue`%o{shHWe-K7 zM)dttvuL!K^S{0U>qhAV`erMA5~^>$`f&^tR7(c+9tGAnVD-ERtZ&jfNH3#spV1dv z^)pa|`ev=eJUQ|FuW!uKo^1D{s8a38^sQ>zlj&Qqv?nvK!J;;u^$Rl0$UyqmThS-x z3GPBiG;C0|;9z>jWX}IbA4dY8MG(+eP%Zoi3iKt1y?z4Uy#J6gJlqxh1GAC7TPxtx zf`5~_p+@knwEATOyq|#lo}bA7Fu_j<)w1Rb73?L)YSDr#s$sx9!G24SVTcUlu3rF$ z3O;!O^5KG0bATfSe>{);9~%%Ygy9KL5G%L=bt>k^3AS%Qr_`5l1;+yGd%gBMpif`5 zr?=zg{a#dKpKpYIeRbd4&c{H|cVPbV0!D$p?Q7CKFrX|GRrZQFCdbpj$)(6pU)R)Z zA`H^^MSUr!-}-*2^bet5-y7xhYcCc(1BG7?1NMFv_@9S>{RE%r2pl4~QwcH*dk*s2 z0DaHYv^$Yutnjr#a?h00&=%r39PTYdT<@EzWQq1dr&y%Wu))F5`2CC z@N}BY*g$=C)`CYMKk8MI=lSzkxr8w6LY?_Qd4 zANVQDk-lRh`2H(^OR~Y&H&*Q%1q11-C^LWKqc2bbeRb7;Xl23{>Pwxr#h^;(zX63~ z{st}+{BaHxmJ4n>5IFEn$anPyjuJdkHZs+3%DDEs9T|ofAVD`;^|AqJf>Xred2fS% z`yVhM>RsT+1J_-(C;-SZ5&!8^b`*w>&i;0E0r_2mj>l zP`Fj_D(NHo_NWPTzQ-BrTcgUC!@%k_NZ)28aOH=<@8<$Lr>+IzyA?1XRq##cAy6VX zZWnM$G30-ieSp5)$!9J2X&-~{e;;t}e}LCZjZG~9-h>s9(=e|xqE5K}2_(??*)39H zNziB%N?7y>6ckVw=U%SwN8C0CSl`jK<$hp&_tR+Es%3qO^exQkocdqJdJtCi6@?ps z@0R&qUkcS&`hdnkUNWvl{`N+|`qHTY8Wn7yzGUjiXz=x=Qp1v|z)lj-w@aNnj2h6l zOC7uu3FMY3<8nt}eaqE_#~`n7v-*BIGSv8jJa*H!VMSene1#bNWM7&8tAr4~914t$ zsDVy*AVH$wtiQoe7ksmvdg)uMeA>c5eS6icr=$ipLEqnJpikd|)#_E5|Mg8^2Rk5v z#`io~8?Sr^3iNeZL$?CE-t1cqtgpbjG84G$OBgWM0bDKknOCU(xs9r}K%jqfltBM{ z_wjrr*t!+`i|x?~^?gcL>OnsCYw$lm0e$+8tQTHDdX497MyJ+SVC6cg6Y;pM*aihB zW?&eGd;|P&8{jg*JE=-pK5jesAt}IV-va+44@?nz zv)Ix-5Ev8)i?<6d7zeDct_qh7=Iw=iy$#5q;&E!fbhk-A*1@S9JCLb+DT$Q9qOZMs{;y6;hfWKRS1lfYCzC(hLdf@lE8MvZ8@O!jkVuk(< zfG5-OJrC1Z!SQqy%eb&1_$O8ZM>+jLi19@)k8T3I@GA&R4FI;?4TV*L`|Jk4CJ_7) z?ZGeP-)W<8_8|D)&4E`@pJD^z1wUH|?2K;#fh%<8!va1*z`LRaw*;imP;c8}J=nVL)YD z;9{nSm44*f^P**=hJnb|Eq-%TnT})+aT~~AK>|M zz-#3|q40KKA8InLp{;yF0fnuzU~tSG!0-JGyeb}e@EPdS*VWwCn#MmX(D&9Hd=4eh z*U6;!M2YwE%>@(&L_lHAXyCqM(8m6JB@l&|5|BPoaOqa0UoZIg?vU5l*!(n##y_{w z`MaUuYk88nI}vzOQz*bWG0RtliFZc~uU(~Z@AZje+UdXGF z)c#xpKFJ{5brM)#D)I2=$WUK1^V(fdps$-*{uB86u8B`+YsUJ_n?R^WV&pyG1CyXX z^eoGc1}0+))%(JOF$}rsK?1;KTA{G~q$u7i7&> zJOlWbV(8xs>@+^;BNoksz{3i7cN%cVmylr8!@y%Yz@R+A2cyB)cru;$v%v|EK>jB= zl+xJt)U|wS|6hN-mO$%)mQbh@On#y^-YtjA-s#97LTXIoMh?hpJpD)D5|QsG52MZd zZYY1_kEdaPz5wUpbHMsO)t}DR#$6(zQ1d^U0K9h=3~KcZaP*_Vzm_0)C2l1{CVaC-HIM4fKqMhq2FWU_H6`JOQldmNLQGFO<#kGV?b| zW9lTFCez%tnw`GWOa5;j*9{6OQF zGn_XU2|7t93YiBy@1m1pE6K`G0%a&p*7^I1@DnFB#nK~A$_$|1d8P3 zv*39sctTD%s$K++y%iZ4{4=Q(QbVBNyx@caWEi#({LIs0py2Mm14k_a|DGej&X^n! z*4>N@au)+TXkWm?NZ&)X=Pe}A_fI|f1o--Xs`{;vUf)wS>t^s{UxU8UVMw2~6u90e zCZ>DTUk8C#Fq~m&9xyLJ7zZr_9yS+koGn;yT&e`?`9FF&bAi{d0?w1u?eKSj-;zV70>Qk`;67F% zc_qcF8;~o6-y@{Ng0*7_{~8INxr9Ctz706}LC6;i_9_AP`v!cyh|Smz zT>l2}s|BALLG^D|(YFv_N5ckH2(}GGhVyoSuN_bNPT)o~S8)2k?}6FznCS&;$CI=R z{Ciq~e?ag$XLl&b*$n|+0kOhz!A)L5f;m5kf=w`}OmI{2{V5gTmmYw8>0aR89vDU5 zzXHD^`cn1-JImfeg0w$CXx1D!<^b?AI(Xy~l^p_pl}@qPz?v%Hx4VG9{x9IXRls`% zpEv}4)q-2eY`N* z5l*3C?n5j;>OA*(-jMTzYqrb{{#Mt zeZbX%x3&QGz5@QUZvto1H%&Qz<2GLqa)`j;ik$0JcmQXJr%UhzcI<~iL0-V6GTHcf z1D~ON0UI16c+JVM?PacaW2I8<4U2(*<>vbzZqd>IUd^O^$d_lDyGfKQ0$jBEy+`U2$V zGzVT43__^3HpArukHob_mpklB~>;A3eJuM z&Ja9(C=}Aqyqmo`OV)VwYwso=vH<$bt0{=`iFDdx;os&Q0R{TLvt`MsQhlXY>OAoE zg%&I2OC)KxB7;Av19Jf~1<#Ws7P>vdG(f-iLw6#X_~oN8NM8};{Bjux`VOF^wNR)p zDJuIPmA!jhj?eb^eU}hAA%u`rk|ZgL5QR__Ayi7yK_?_pxhkQPbfO{&rK5zXlq5w+ zB}pm?A&HI>O23)bn*Hqk{qMP7FMEI9v(~IxGqdJ!&2@K=4miZ_60)}geY}i_w~mxf z9blIknUP~S%Iq#7H4k7D{Qc}!60}r~mmhGvg}9+$KfGPD8wzCCDAfM}huCk(FWiJJ zguf|&cQi1&v}BAQ=(C@pm%u|TcKJvdp0uhDUQ$AuPx&Kw#BMDaX8;D-tx3ZUfjkax zHHq>97F!QZ`kXwP(Hf~E3zK6y%90yEK5H82J0th)0J)X{k<<0BUjh)L_zP zFc3Nu6yml3`#uJFA97d$>Td=4ybYjlWDHy#3i2xC0`h)=+IQXt@*_Cv8n@y4Z?_T@ z+_rx<4|&|C1*lsBx06#c%Gt= zV1AA#X$uJ(Z-Bu#WIpnIP7Sd8F!+{(T-6Tr(>8-V2>I1-;231}V&Fbx)i_}Jy^!C1 zH?>E`pO8Hy*pQdYv_N*DMj<{-dZB!zJ;)6lK)+2DT1d(f_#_5pD{}lq(9d=P`DHb5 zpw$^T|2knZ5_sABAVDn^GF0sce&q`p;tv2%B8N!SLG^>c@4tdP+Z8w*+4(TA^-j=t z_Cz_^Rnz?9KENYik@cS@iAh$|^)iLfQ zaDW!DfIo1!C$LTcaJLp|f4Zmh4TOYPHjILRZ(~#og#izPpyBC70;gXEhjlIhOOgEo zt&q-T;6?kO1>_TfjmXqZ%acz74&dithhmZm2}h=YhxYWUEb@sVZ^LPZ?1#YRbAh9t z0!QPf()k5&*LaY}zX6^>MiI@gR|o7pAMC5w1Ha{7Ktj}8NO-stY7o@`+(r(cX^q61 zfW^q;G-bhN;34EI*MoB8C>mNU48=VN(0;c0_H1+3MBm=WR7RK3KU}gfY|G)BTe0cR32KG}tK(0BQFkSz}ul`p8 z;R<B!ouOOMIgG@ewEKdK;XKH~nF@KsNunWECjeJDNww~%!rsea}c~F2l zy?2#-JV}RA1;$J4V*~MgeVt?17F+*tU3jFC_V>BPURq> zXdfu3O94+HPcqa%+Fal;TwIz}fuC-MDE6Yao08ApHq5^SI9vnddB|FMAg@{ha`Ra* zYK*y^kdTVw+|2^mpaBiq03Y=OZbh~r-;kgV%I^jFD{UyS5!w9?aH~DY1-=1CI03Ua zrf~XO7GRF@GV*9e*$P|ingwu?HyFI_1P)dp`>`h}ACTW951TZ9 z#4%tY_`)K`)A56Zq%bg;M}9ete#Dmnmjnafz-_r-DDY$Q_)HBJodMpBN43h)z&-sS z&rAfqR)$fZ46N8qV~)gM1SV9nvs>hhrx27LDUhM)5*R2#HcbL9NdqbHl3*7*oIETpT`X7P`?JFQd1oCUrM06OM+=L91B!T0Q_uvbPj9VZtA)8F9 zFL)bx!A@|{0D0I?w2%B_A-?}_!i25(U?7jHX%VpGYsB!O=9BMeK08V806u9 zL7x2tSi==#=ozrAFa$Nba;|ST$l29%qU`yf-97hcJ1DR_=qjTih}a!}K9sxIDA_>MXgXpujG!TS;D<(Wn-G1vTg;eM*^KXs22O za&kG|;3%2&85|0#0oI5Dc@n+&oP1n2LklU_0gEpIxo!h6*GgVG(I&Qk2MLBaH>Z67 z{_6z=2z>^Q$2}fnuDA~m<(R~NXq98 zgkvAm3-icl-x4qo_Y-){EN~zVc?Irp;(9>7^g1}CNpCwRpF%~{M?NkJ^48xVmq9lE zgX{kj5l}E83G`!uto#>vlo-ehkk^d>_UHq-51q+K4b}UB57~eNz5jq`Sp&!Lh7H>H z>HvMIA;W0>dAu9KV9*!uHr#a>nB8?49E?qhzq|S995A~Ram9LIcBkUrLTFKTx8fIE zeApd}i8vF=597cidt#9Gkvd+&I0;`K_aTSi2Bl~?=tr7^gC?VZw~$jZ+5%caz;`#` z#3Kxx-3j(}#DL|<7LS%!&*6=T4i3;n?6TWgEm#9_$S{#?GHC{#alj7UpdU09c;_47 z0x94Q9F`_Bz#jM(tV9-AZ74X*HDN+rCe)}I*@7HI(i*iQC*ad=fjk(nB*$`8?lucJ zn4D-(ZbrVh4!Xom0pzAQ@pvIiJsotI<3-E{h2(Omp|c`zv=!C}FWk$-^_^Y3$4}yh z7FdJwz;&Q+Fc0!C!K2uq`M~?gR8IA)RDq8S?hi=(#i>EU+jKCN(D&R6`t`^KX262WKpwRLDinnL=@@zbpa$c$K*6*E3W!+_ z{A3Yu6Y>%KYPQrakXMrj6PiC`H?WfcI3z=V$((#l5+Hw3FmN*lX;cWXwkLW1pau$J zAc4IL7LNr^kAVUtk#)`h%Ol$a0jnZg9tYM!79fxJ)V?0_=)=Iq$SMa=!5kAtIf8;O z@|fEa0Ifd zGjJU8COhCHWQio;Okj@pDWhgS>$pt69j@-vxyUWagt>kqfM{8#3_nfF;S( zFO@4JpVt7n3v#&#$g`2_=Yw4E9^^O0V>wTSb=b`*d zI&e3#1UZPLa^8K&9~}zzqzZu7?7{rVx<$Y;Tp=iklDU}%nLTonBN&h`2DzdoWKc)0 zF$X!j!S4n>8+xJqQyl~?xp0qoMYeQ?pdW*LQ3mXB?6$qdWCKELz;4}}Ngg~XH{&gQ z^NxZ+-N#@s8Am}>39tcqgF^L<=*52I!~X7<+ypqtspJfZR&em*Ka#SVeBY2T$16gG zjbz_W6~vxHh9CHBCxd*$6$((T0lA|ew3tghaM~^`;4N^y6wCt|$RVRa->L!R&W9+I z_}6;}2@O7Apb7aIIVhwKX*Gg;K`j`lL2f61xJ2bbA3?r90`l7%Tw;Be&FuOW(Ob*0| z*hI)bJPQgWS5OW*#FGa5#_S5TX{7yWfiaj6LQXKKL+sA8+FEDpK}30lx#HL%!Dkn5}&WRAChDvJ)GQ}OhW&F@0MrOc9YS6?5=D>%^-}TTSW)~q{w1x^b;l)UiL!bifga_Au z?=z6lhYD-%05|OhgLy6BK$IWdtw)X>GjYllvfxxu_V6Y+_SZp!qt40Dhk;R5O*c}D@ z)eQ8_&jU-~fd!`%0|}k%R4W$P@CO)_xCp$>8w{FX0dB8`pwqhwY>G9KPXu-+C!f@T zIONOyke~69*MrRQq>>=R+h3r-*mW`*K>6Qi|H}gSnocSi@`uD@e&j3U@qEw$Yx|T z23ZS_Vw;dZyW#nNYXM|1#9e7`G4RRTP$2Iaup2%H2vz{E&;q%^E8ti2phZHEFRnm` zt3hrO1oB2?J1^4zv<3yQL7{aKG?91>aJ4o}rJBf<( zzzDg#4fKa}4$@f5?_YL^j zHE=+%b&xq;6An8+NZ`F50U1KRLx$`9z*Zf=@A1!Ss(u0cy##}$zkyj7IQId6FNFMQ z$a3hAsVD$kwhQD2 zLy<8i3Xq$gf4^Tr6$`ja;xZjeE9EEtHw=l2j~Ih+|Ykvna`ff{77i;!Pj9P)3%Z#bAE ze;9);hAcY?^kv3#kl`j-uj!V_Z9MSW)er-X$O?;rgC>A{)HGlr31EGDV9ANV(~p2X zV_D#I;xM(JjXW9W8D&lp6moT-My0EOZ>2+qcI0(r-%kxVuLgN(70B&%v4Aq*G~_#e zP(!h`AWysrY=q3iXG<;$6TGd!fePfaRxtC`BOf9!45$Nf>%ibxWhhWb57-*tg0&+D z?gssMeUP&l@LLZ&Ib_fwj#r8aGM~VKUgQi>@K|dDWDqACh(QOC4YYuL4M4uF2l9s? zrw#{B+6Zz@Ip9`gi?7r^iNAUyP>A$~0wp&C|6B$^V}dL-66Ai!0eA`)hb)S_+)`v^ zFK|$A3)))%?1ij%g)m+JmA8UI<{2m;138?0)0vLz6693@Aa6vzK%V!hya{=pFUTc~ z!Jg}S$S;E|@DAk4$bN6}`~P~F5OfU;Mj+2n2hKz;QUZhR$mguVpv*QXZ~>_eb;umq zhr9)&9D_VSzOpgsFtTAZng3{l>UJYd57Q$g5vMCY8Y5 z$gi-%pMKY09=o=FOa+10S z^taGACWHm7fw|ufkf4YO!|59jqF{w=LEm@~PP2gwhvI;{k&Wpa4VhfPt&@ zr4h*>gM6O8G$L$-tVQ4H5OzWS9t#|We22c$A#zoBu%{Ia?1em@zS1G`0(V^h-+Dm; z&jT|2I|8hQ{EWWSAsM`ouQ~xsdV>CSTVPG(wTZwoM?pRn9gIL8jSf~F#q~dizGx%` z#38ewsYhmg+>6ZmSkVg#V0~JTkd>bg;#InH>AL5t;RIA2RC_ zL0_n`vH`|G1hN9gKnC(;jG0noHmdrNSr-c(2M3^wN&G8g z!hiY@nKhXkGHanEWY(u8$gIW0PCx-{6qzHlQRIO<*yZ%^Pogk^d02$Zf>7}!7-W4K zfy^AvMrIB-Av1@C{2)K;LMde0g&d7~159A!*aVr4V;f}Fr+&z6*p?u(VcUz$hOMMO zwg`5a9x@rW{rIE_Mf#d#tgITHK40ND)ZfmY;-s}MsH0pLKw1z0V*-m(!BbE`HGTh2YAA!8LtnQO zHb-VH(2P8tzW*n3=U~Xsf-Vi2#aIt+Wc;%lXQKk^<053%0u{)thV{s-hRw)qlyoDr#Yr**D#RSpp-k6b zHty|Ffz>bsnXTU?$ZXu#{~vu`C=|fv0ZC*Q1M0}E!X|_V*MAnZ&ZxkmHVSz&25CI< zLtHI0kmcytYDk}#A`4swZboKPu|OC&$XZl64A*}a6uPLuMu9UjYaw6c|FjS?Tb|>Q zS%un>S&N8=g9FT-ZutLM?_E%V8H_<@1~ZUZjf#+25LO_wAg)JdUC@loMqM9=39LY| z2=I^k1yiUxz%9ifj4keA`|*@VnmMCcsovnEzX zW>d8}GK+}}7sOkm?W1evu! zHZqI)8f4ZbO~|Z8gf2n>Y*fi3vzgTdnPW}nh6!w3M5`oLWtP3@fSr>|3 zp>i_*+2Rv2n84$)Vb_Yx`cN?*46;5nKxRSdhRnJs2$^+J5i;un^#o#%uD`6wgJ=Tz zu&7N#W?hht%(|cmnRP)0GV6kRWYz@&SHV8(0#(9;>p$xPEmUA#V2}KtQGv|5APt$V z=LN{DMH`XXL?xdH4zL!`O~mz|1)&WpuoiJaW;OOgW;KjMW)2r1vlwVbX3vgdN#FpR z2{n^ANMLmUz0MAv^SOkjPSgv@5XDrDB=lGmX? z*5n4rEb23nS&M3AfIe$M!5hFd$OqSdc}!q6GC*cgYlY0B))|@AI0~5sZ8kD%fg)tq z0=%2x0BHeu{+GlA)&la#Y#z`>W>Fu6%x1Q#}q@5A-q2oqQg zxFEAp5Q5A`K@2kMq9kP21yY$%APZt&WVZN7+y?o04p%dCOkhD*fy_1(0(T$-JMqv( zPQ(3!2{N0=JdoMM6obrKI1ia)P2Br`GN@)jf&b}qWY*_F$gIyZkXav>BC}!Fip(6) z$sV-N{b%(vhlKyEeqP9|Pt%auC}>7z6OB*~ILw;d9GN|SME8*FwZjX!^bl|i^0NKF*~otOz*Wd`_!^IE$AmL@j3!h79-Cr;ipa`2)V2_-LP3VW5fK8r;%!dV(BDY`{)iZtUl2+s!c!nfY1P*;q zW$QmC@GywtkSAe_lpvR44SJDZ;_4?}3|GYJ*ph6?^Hf-W<3{40#CnHHI84+-_C5Q{M&{tOHz<3yy2di|_>Wj>t10p!45R83<{1qpTB4lu|E)e+$a=}X=cSjaM zhrS}~lRX{HzoQBC*IWZ0_X#)y^BaE#E+z8D*^gz04Uqs(gt1?N!Uy~v}m2Gz(X z(BS2-pdS(j`XgI_6+MB~z5z#)JtVasgS_!7S^sIm_*PJ;Amg6$G343Uq@R(4u#YXj zgMK1e{iyyU% z8Tfw!UnYG<`6zM^^7!*Q%YDZvaWJ9u=QF$D)0*wj6T2i1dLDqB12FN$aa!xq}Ig<`c!iLj9 z|0W%_guSE*)ActDJ-kd4$Y-YqC?q0tuE0si64*k@a-eTYhb1w%5qU*Cu-HtH$A$ps zD*!K~T}t$gW{-gNe_kpi+))ID^W@2fDl{PbqeGMCfc!6wVj`c7OmDTP+>1OM?Wrq) zejYk3H5a&+1|`wgoI7H0|G_&*r&5w&fC{|zA1E0TcwkT=nJg2<<+fc{&u-=J))!a;%*ohnI!A1Y|l#f5M_@@DLUE%QOYFBs(e zk$1ZTpGE%V1e}9>5`(-Rc^92$h1s-t6Hx;P+%d}4kw1_<8dcbd{D;nzgij#fCmR&Xw~-TY zRJ0?@l5f3Ic`xzu)^uhh@*HFpI#tu(pAc1t3^&n%7pkIvz$Rgju zVfqSw_5`LpnnBJc&{DsX7c-i)cy_!;G%UPH+KZSbCk?KG=mK$Y_f*}U66l~zvriN zKV%Je;0)wK@&_TbfOaP^*hRiAPxUi=fQ#uwM*eKW#(?}KBKeG?F+jL^6R=$XjcSs> zGlB#ay0{RQKt4`qGQtJOs&sBAoV^9~KPCgWBR`};Nb_$8`6W8+2pc1N&|ys28Tm6u z=W1#Y69nk6BV2-f9Ub890E5Tr;zZ<)Ccxv!5e#KPQ{W=H!y(*j20W6+1YxnAz;%JZ zUdUV=9j8Pg1QTNER7^MtS(Pp>gbR@S=(s1WYy}2A&;c#vIdl{dc{cJny8aU`LGGhF zBEpTp9Iuh?a7aQgD%_&06Jd$nP=FL&y$H)A@54UTLf(yC;D_8xHzY(q4!MR^0Qnkr zapNG9@qY+gpcfU&=oX9^5ZeO|?7%LNM^>jhCL*t}0r@aGPY@Qg1@=S-GVFlOvB^u2 z{m9cY)z5Gsa=QK*VGuUc1oH90CYEvp&cUdaa02$l8Ym-|V}ZKJ{#c+fas${|M#Lj2XDHxw32+qh_r=&E`#@f|66Ai!t=PxP`$0a2uAao6F0v@LSO~IG z6v(v>j2bMTr^kZs02tVV2FwovOJV_T$a!?fL<;CdPNvH=pM=qfI0V3Bz zzK1Q4eTah$3Uq2E3KfTdL(qX{WG{@`UgUM@SfCr|FT^HSMt+2HFXYYi50{92iNm1Z zi9xN*VZtxEm=FaU2gUDYLL6=4uxfdfRZ<`1oC2cV0q-3bU#7#osl17)cYaN zfG+2FahUJ{qdWt-1e>TCxgI^P@B{~5&`l^Qpb_~j_Hi%rYMcp`kAl7rI;@MVLiZ0N zN&J~(LaQwpbVGK+xjY29@C3+{kZ;i~7IB~i`4G9+1e5k$G(KK)#AouOG4x-2)Q+IOO4IFB`c7yPyJj2U$xvnxPpJ9^x>Q z_kt!{jzOn|JOU>kW5#p?Lh?H!uf;B_LB6vWGMCWAtt!dXE(x0 z$ZEKHWh3+9IIlqV#29Eswr4SboP9(5I2sw)O5n(@M zM{LqK?=xVU+p1buF^17wK8gpp`4133r#tN{52wonanG>(#X z&HHhD903R_Dk)AcVH=Vq}};NdvTAdhT@U805j8C$>@`6ULGGja=#0zc$^I1@%8 zzo1W2#NiCWgX^!!;8aNVc&IQDm&ab@6S##E3kCv^-7V>qJ>mu(*UV}VwF6gfX=6G9{k;)N+2B@$}3KYzd&Hn=1Als5TkLtT2Cz_GU zli#+fK$ar0`lfTCJ~%w1PM}=&;mQ@->s7j^2i1cft8V)b^#k8 z)1RXeC4UxPszCmSeBP6mrmQ>9~7NQVEH zbyi`qkRw0;TaPT25}{ar>DR<&4f*$m{HrDZ>d3!(^6xGA*FgTg8yQ*lR7mOUGa)Iy z?#L<6gkCEUYhDVZsQ>?2vuvIVnJY!ES|Yq5%eGG_cSzPuKH(cPvbM?!uSm|4Ef?l* zjkG=^tdNzSD`IFI>6;`nJ|rjxdr<`!x0 zBRXM3FHh7nLyDg#6#hU^HZs9SbT(JG`hA>CK%LJ$iM*4$Gj=*k2+v~Rd z<&^Nfn>XYpM|{2_b!YRq z<$?VM!G>y?ix&@lx-wl)_Q%ep!*^f4TN>^a@=9#)#n+NDI}=mFYf8GW#XakZ<+?74 za2s9sn5T}usnzbUHnqd4KU1cnRCTD&w6~}3-`M^nV*ku92R8YtTu65)Tv8z5@p{d~ zqnCuV6xTd{$X7V`&EgH7Q@0ey^cg%n&{|QP`)HB<_)qKlkDhY337Qr<_w>lp^T{?( zcAQt-7L+{VnDHvvk$0_sJvVV5v$A$lXQ}Ox!cMy!PRl>%ib7OdneNp5QpvC0KA*j7 zouscF9RISRpxjFCRFk@KY0QrlgNR*mX+||oGUd*SAtwi z`lWN95>E6h-3yI;MnKvVyt>`!w7LIuMeb9+u`C#+xp zK*GB;R>gJhG53LgR(I6<9j}K@(5#M;GCP0t>1A1GC(~CG@@;B12=y0<=|~K>-?GT> zPwSaS%h%oE=BKYZtTwmQWHHyfcf`jtJ1eY%E5noRX7}00RXzM(D}PxbYDsz5tS!S$ zRl}sqbKm9{goT!;h#sBVp+2`ZeDd{zIiFK4#v2Wl>EBRyC}*C?^Rqo-QDP6Q?&kc_ zjO`8UI34!t%+HJOV*WZ9{*@_r+S~Rg+sw9k!Z|x{%$z|F0^C& zK$m3glX{o3na$oA9uYN)D;=kG>eS7-T>9Y7Q(wpWG`X|3<0B?r$scz5-14{8olDk! zKHDunu*>C>`J!h88dM=uy<^l9k-2Jba_W2-{ke32|u5=$b4NNb?1%bl@qNa z?M8mF+}5z@hRhQerEr@tHNlW?JsR)%%_hmGqzRl5=}W)#rKz{hAT~jFtG&$ruVbI= z8F-{`>CVkM)Zn+ERYLmPwtyu8=hx{cPY!)FJ$b0Oe~;9kcQH$n&TXB&^>uIHxP%YQ zx9`Xrt#z!wd;8OlxbObi9>#sOi;h1wcqG4a@!bniv8RW*R}}hRbMZ+Q9VN6Tr1Y4y zm&oofXXl;^S^KVLr~QKQ?l08)^al1sx7kU@rCxmX)%w?@1kR%Wouv3!=jP})Uykh5 zj!bMcJ@oX1+U1~KmH!N!_1Bmw_uI-3|6r^vQadH&qkv+?f+N!}94cS4&tvNNg}i6= zp@%0eb{NLru*-dz*)s#nRAGzw^*&*3t}6y=@)PF#sVdyObZ2(c&pn6QcB|~xyyKzq z%INbBt1k~WM9ixHYdklM`z!Zzqh686lb+V}eXZY>m6sV*8>|^y`oLJ>_>>1_%H~&o z3{~FR&+FqoGR?hz^xPkfWw%ww%B~evIW(~_)8Xb*^I3Zh>pUcsl+<$8|Mik87cLwl zsoa>ak*Si%h_8CGP6x-_`8&Z{*mU&XZC`8@ejeBt zyWwr1-NyY%^0ySZuKic!XRLCXo%Qsq2N!D|VQ#By``InXZsiuW!**dBd9Ro4Idt&1 z|6SQDGo;#nRT}g@%v3$UUv^X0^YyI>o<_&Z5AW$&8tSCC&_QH&{^@$RGHn==c zjlHIF>GPg%4H{AoxhtIAWB4sH?~MObG@-A1{_lUED}pBk3w*7x5fHVmcgmf;r6DkE z-3UQW-ema82U9JFkF^ePzq0v`Js+RMnrTL}bT>adcp*Lh*v_>^UZ1v&T4UXReZ|$K zdQPhO@5H89I@Wdym>j=x#VYRd^N$)kH2Kbn-+1F2)#Tdm}^lG^H7R zvVOKFDYhi~dWZeHO?&??D>FHoy-Y%3ZOHV_6+0)oCX{h^s*VcPTl`Wze0DWo#z*TT zj}4b^4z`wj%|$ ztsUd?g<&$!vE``^8Lwq0#;ZwHJYw7gB3)h;5@Cp%=d`hCgKAIB~; zpY0Q^qBJEsVb`tZ*!l?Bt6~!?zs+4UtD)#r*zM9iBc+wv{KI-r_#E86av9G{Tl(de zIM1aI=4gKu^N&2|JpAb~^0Ub1bEFLYgKS0&FH(9t;g6t;*x0l44h{)-;|4kwd>&?` zI^AjB{ZUKa+x@t*zJ0%U)oz2-Z%J z@o+(Z=d;wtNx!G&+3SeRtf^Xh{kTZh8nq~U&&#S`U0la|CGxJkc{id&x!p`_{-uq> zOWvp6?pzl5Soff+uVTo{4bPqV=SD$oof=*qZy}(JQbJcjCsEZ{QuX9w7{XI9kC2M}g_sE-PlX9$Qo(r}7Efg@zEnIeH zg6>{!Tzzw<8Fy^)GIRgLkcG0&juAhyn{@^@yfGDOf4k|Y(w0>VZl0(vYB{#}@QmMM z-fC$L=slgr{pb*x{^v`{tiHzw=NT-Y*x6AmU;0-zvn(LecVYfF!%JG7cW*@I-guWf zVMv}~P`27arRwmg7coBrmOiVCdE-9myzA=zk8cxdFLB~JPOA=OnRnf@od2@aO(7;N z@k`x6dcqgeohMv||6Ba+kxagVR#WMVxmr8loKJHbd1hUSYSOiB{_h-?DabZWda!BH zz4j-ny2|-ta$|z7Y#JD7iaLBN<>1XOvwInQVOQtc-Wb~wU^M<@nVbITe?>N5ULVwp zdvbMVsGF;HhhAqfS1dN>{H5Lo{fb!23+@^B$8Og7J@EL7-W8jm@1MUU7kE0x)*Bp& zt?#gKxEQXmyzh0U_#f@y^jKF5sg*`ClRppr^GNZKj>@9T4W@!AnKQLdOYX5(zZcnK zBYwj!*t9QVY5iQ=TE}61$>QIw7Rc+g3ACtBROdmbyag~hH$kYjrIwWqqO=jt-SM3wqUF=d0ob}N@r5}7;r^@Qw2p5OS z$xp53TUI;O^+e|76-T}_T^Kyi``FFJ$KEWHF>z_kzNP3Se>}HXQ}c+*Is4yv2gl^8 zJv4jbn&~hzP5q3Z{!=-7JF}ilD<4dLX@%|P`3~Ri<=vRT$U9;u|an>T96UG>v z&4_QdIQB2`@i?b)p{5i2D{kEx-j#Jsdhv9lB_+)ok9W17?QskU_Yl4IRNn3F8P2|^ zZre)VDOPdGiT`wt|NZdVTTO*CUr_vh-@eE08g%^rXYQ9>v#EmvT)<@^4hyFz-Nq+x@9+UtVtM;wlm{);d{N47L2Z z%&xxG=D2?AC-cvHW;U;I5V@`{uYTf*tOs{|*NLt%&XwD?yo{||{juX6*D+~$=kY9| zcm20oC)N!KcrKLvhhNak;B4V_IY+H85e}O>RhR9It!xvHaBWl-G@X4grt!4QB!Q>m z^4sI&XDByiJPuS1*!|h^<_o#l;ZEwEMXz1d5|5WuwFjPmVE)!#w8B`&@B3lDjq=uM zA$dbNd5yz)S|*j6qD=Y7eGU@p9Fr9N&knT=V&F?h}{CBHdC!{W* z)7@e5ka-_Pngqz(*mb(7;K{;(*|sN@D8-JzR<%4Cc> zViL+t+z%A4`Q`f|GVJo^ubnBUP!oBVYeo_dd^;<++qX!DcL8+kC8}7-M;wi*>hVsVej147lkz_LP$FMT+{=G`Y(tlsp%MJg$(OG)CtorpYnHNfg58iw_f7kfW-NJu2 z$c~-<>4DalGD9tY*^{d6%eNmsQl%1l=!c7SSB~$>r%Go6;|)9n+YCfJO!X2!W!S53 zTE1uHyFK5PikjZq9=DhC%(7D&n8)o|bl82jjATfTJ!kZ*K52e~*4ujhtz*loq;D>M zD64czEoptqxm;am>BQ~H55Jnm33eMFAGWB~NKNHtm*UkIYBSF&1(ztV+MqUi7E&-`sFn^3qTxX2L(02Sc?iS5>s@X>3${yE*0Uxg!g8_xKm?*>Nqw zQ^;?n)oIK18&;Q>YnN@=@ylZWj8Ez7HCm)s7+;tddN;st{VBauXTGhnuz3@!l|0#I zd)y!12CuvY>s_biM=Q2yKQCADo)s$fNIv*LGM&uQ#y>+xxAwUuePFrFLNtTW!S{JDVckzVntnPv^f6HaMt5A!*ZI_;1}kK^>R zB@c&<)Dq9L+UpXq1@!bGx~7j)`kdk&}25D)_NdCHGLL&AS6D zy{#kf8s#`U3vi{{C&Lw2SkDgsci!_%L(!P@nLQ?+w!bDil$5<)9kp;$_pG`lp=y=m z4$k>^>*=DIsgts%jhZm+LY;hB>CWkk`d(>2oqlx4xTwgqD>Fr;u z)RYv=cL|tS8qCfrF}QTlW~I#83&9)Zito&cGqW0>QNK-ZgpQbpDR=E~+!0>=_7{;t zi&xf;Pg_^`F_Ncp_;SuX0r#yf6NmXdsn#>U+^k}vSk=EGI>Fsv)l(v}D_hz&LZflH z!W{wsCzlo1Z#`tu6j0k^mgs-7c(Tc^!(8Rwvzh;PtrYrZ)AChb&QA7ZSC;J*k1>hq z4k`L&>(ss-_8VsM?A7DdFII4Kymyo)pLec&ZC3FxnE!o=-M5JTv6-s3b1L6hXYVMu z|M26i{(p^4_b%F46!`T#Sh`|!>F5l%U-!>DiX3)JJ$Nf3M-Q{k@r4bjeXl4%Dy)l z@s|~QE|2XjX-#SL58Z#@yzCU~Ftt!Yp|NdTRaMo2*&gGDRnJ+y@=g4cP362hYRfb} z8P7FqDxNJ_G|5?OO3rr0$f2ulgibU*&h`8$+a=n*d(My22HR$fq9b2(##yB8PmB$1 zZ~Wb3vgfm0#02e!mu`HrJ!b!=N8HKqw9~dHtGn-Btr6MNzHN8((t`N-h4&{1P1YLc z7})D!C&u|$&7N)3`T1@26Xo4YJ*^w13@=8jS7jHx9QAYgYK^ireqBo+Y<_Vq@}sQk zW7lm$u3!A$Y);UsvYB95;b!{hRdKxBd2MfHzp_6c_b!Pl7hRkA$#3DlfU5aPe0eH+ z{4BQY`#dW`G5NWDjqJzxr)J`T@^79^oZ8dbXgai1O14d$voL&XCAYeud%~3GqN}Gh zikSY~6TanH`L{Or(-I1Xekd{9UGq`Dz+38M zU7vLB#nt}Zb>qy(ufF=>ePCUda9En(;VLZ+ljENR{T>vSIT{rdDQ#P(nV0IZ(bLKJ z?EQ~z6G}s`4Y5p{XE@@f^+;{?+h50dXKcEDU_`)?g4C1Mzf263Yi0B}KjTKTBWLHn z{xxw$W6G>2Zlie256aBVpioRP{ zpo3|4VL@%}sF*${zguoQJb06Wrf~yHii2$n<&W$8z8yOBnqspI7vdZ=@$l0j)}O`N zr}^ghWm)Zi6?*%T{|VFkQnxP8O|rcZa!@wx>aA7oUp*Z~kFB~G()7dObhYRCj+TuV zBW*q`-=sdpf7bKHijZ>`%v6qxP8&L9qVD8ZO(V~qNl{q!txI&XSwUv`Zg-i>H?6JY zxb#c2BG-7xtb5rfb35GR+A^&D(0d3h>euKisHPn}C!N90@?*laucwAzH;rUA*hU!~T|o~T~^q@`uzp(8r| zvjbWV3+!C0ApT~a#@^36PYTa@P+0hgZxp|7YUS!zryn<@d#rPsSE8#Kon`pdeckiI z_V5iKrZ>8(*ev<0%CC~Qa>tEr@r(6bBAw5PYH_hj`Xyc?9xTZpYCT^}JypVJNk~lA z7-t`Gks;$kj!pg8)_$sTZdd2sz>O!Gf9B2#b`Eg7z1Z0~#sA99`AOde7v4M~85Df> z!W19WW#^nhRN50BDlDYre9R_@gc58Qn2QE_R}4{1#je{#9)!)|0wYa|P; z!N-4h&W4MbFN<<&36J@p7?LYM+sT|uiQTp0+{b>QK#>8<(L+8JJXzy)T01^}-bGtU zKGmO#_`ato-Zj|4Kj&obrld*VPSsv)I%~Y;^F~dfwxvmDL!viY7a0k1eD30j{S2L7+jsKa9+|jj!(wH}nfX%7dS-P^ z8X^A1S;~;FK5LzJ_e_b4u6trW9n|!Y{WQ5mup)5v^ka9fDF!~cC}d|pHSNyT`v)Fc zX}M<@%@0?4c;?OcuKc$Vq9q>=2tSBFRcW+huR_rh?oZ&*7oMBG-Ip;r^-<&8X+iHd z{WI%+cT~EM8sGW%ylcL;$fI-CQ{BU7-#$0$O3#eEE&TesA0{W9S01v2J9z!R`_K}v zb^88~pFORU{9-$Q*Zz*Yy_!?+r@t`i2rTrCd?L4IuJ-xJYh(+ZF0C1LF*>k#cB86l?%_Ls0xg2Kq}vX-C21^_ zzcr=A@>1X5@~{Dk@Mj}i?^&nRE{>ewJ;}G*|F*Nfp?B)DSGAd^{w8w5s^T39l2Wsu zPd&o7dGoxz`{Yz@mdtu`c6Z_Cmky^rTwf$RzwDYm{H(z6C;A@JikA=9{F^&6@#o7k zI>wjxd&{oed)Dy&y_06oQ+FDcmiF1MZIiPTeZiOiX8HHlf0la1PGfEt=j(VI^lV!+ z;G#ZBqGbnbcDC#8oe*-)^z5w7>1f&T^N)Qz`cbzMGrGZQf^o@ksBVTHzwu z^Lu1E*M1z4b$^4$rjb3~2CrNC6gCVy@;30wOxy0lmOkf=YATw&z1LE{G~3QP{`IUR zSNSmG*V0|WCf_GNdeL3DeAdwz)$1o7ZriH6>&XOF;j21EtMAtK-p^?tmb&xcSpQcq zCx=}Z(q60Gw^!=&{)QR1hwr}K_FKRu?5|vuQv9EM@xx6gS2S(A@Hi+pFQZ%9`u<3* z{%@8voC_>e-w5f0N6YIbaK zSZeBZ?_9<@{eQ|sH%jbTu3zVtY8=4#s(9j$M@9Mlf(sSC?>5+`n=_(W^qj(jcY5#J zq9>g5b}{IAmG$oG6O*5Qo&OZO6#A-$Xa_#&nZVd-%eq}LIh^~s5ZCjsA`V7N z#B5Btw$N&N+V)M`zX@*2jCf{#RHC)yjn-9tiMHW&x}mc_DPEd7B2+Nv$N2F4T7^UY zl}ESj3Q`rDS}<&_Mtiq-%|L^m{na6j{69=;BP^F_U)^(K;F3dv;j8Z53Cs3udG~mA zt(;cfg~Mi_ZpBISJ1jfeyeF!bGo979D?e4?@sD2htGw0T1`3DQ46Ab=f4=|J;pJCL zM>RbYbL-d~e79|Zchsm)Ws?);`%Tf6{=7x!!WQY1lZ|HZkC@ohFse$k+gAU$&#{}i zx*zW>tco|+9Jl8375iwBg?AmhG}^!Umn;aI9e%{xP<2@PO>@hRnP&`II&Ci4yf@~! z1z+1V3krHOrN)f06dtWLP`gNTn}(~pY@TF>>yWoELy!EqzV*`eY%#G-P(1(oR*pT~}9++exp`5c|OUzH4}J$-LG=hMu0*HYw7ZhyLEd;Gzb)2eAZ zPl;5_`|3Dn2giLo5_4b4uchH`nCZfn(xFQ)S~M0_1aGT(HPH7yzG(a5j_&&DL8j7T z)3ZC)s<-o!lrHTE(GGhQzJI0k#o4DvPFfZHWxT9M?)8%zg=TWz`q{>xRo!xUts%2} zzuc75kTcAZSbgVf%Sz$a3xbmOIzLH{-u})pK=9p?t-XQj9 I#G=#x4+qA8kN^Mx From f42b377f7efb7a800960b998b8e7d102ae1a3c17 Mon Sep 17 00:00:00 2001 From: yantian Date: Wed, 11 Feb 2026 20:50:43 +0800 Subject: [PATCH 3/7] fix --- .gitignore | 2 + paimon-diskann/PARAMETER_TUNING.md | 9 +- .../paimon/diskann/index/DiskAnnIndex.java | 99 +-- .../diskann/index/DiskAnnIndexMeta.java | 66 +- .../diskann/index/DiskAnnIndexType.java | 4 +- .../index/DiskAnnVectorGlobalIndexReader.java | 204 +++-- .../index/DiskAnnVectorGlobalIndexWriter.java | 138 ++- .../index/DiskAnnVectorIndexOptions.java | 51 +- .../diskann/index/FileIOGraphReader.java | 293 +++++++ .../diskann/index/FileIOVectorReader.java | 223 +++++ .../diskann/index/ProductQuantizer.java | 396 +++++++++ .../index/DiskAnnVectorGlobalIndexTest.java | 4 +- .../apache/paimon/diskann/DiskAnnNative.java | 70 +- .../java/org/apache/paimon/diskann/Index.java | 16 +- .../apache/paimon/diskann/IndexSearcher.java | 168 ++++ .../paimon/diskann/NativeLibraryLoader.java | 4 - .../src/main/native/Cargo.lock | 21 + .../src/main/native/Cargo.toml | 4 + .../src/main/native/src/jni_provider.rs | 727 ++++++++++++++++ .../src/main/native/src/lib.rs | 800 +++++++++++++++--- .../aarch64/libpaimon_diskann_jni.dylib | Bin 1024496 -> 0 bytes .../org/apache/paimon/diskann/IndexTest.java | 55 +- 22 files changed, 2946 insertions(+), 408 deletions(-) create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOGraphReader.java create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOVectorReader.java create mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/ProductQuantizer.java create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/native/src/jni_provider.rs delete mode 100755 paimon-diskann/paimon-diskann-jni/src/main/resources/darwin/aarch64/libpaimon_diskann_jni.dylib diff --git a/.gitignore b/.gitignore index 3f42fdc44a97..400cb9104d7e 100644 --- a/.gitignore +++ b/.gitignore @@ -37,3 +37,5 @@ paimon-faiss/paimon-faiss-jni/build/ paimon-faiss/paimon-faiss-jni/src/main/resources/darwin* paimon-faiss/paimon-faiss-jni/src/main/resources/linux* paimon-faiss/paimon-faiss-jni/src/main/native/cmake-build-debug/ +paimon-diskann/paimon-diskann-jni/src/main/resources/darwin* +paimon-diskann/paimon-diskann-jni/src/main/resources/linux* diff --git a/paimon-diskann/PARAMETER_TUNING.md b/paimon-diskann/PARAMETER_TUNING.md index 61e0a5126112..9986cf218444 100644 --- a/paimon-diskann/PARAMETER_TUNING.md +++ b/paimon-diskann/PARAMETER_TUNING.md @@ -92,16 +92,9 @@ DiskANN is a graph-based approximate nearest neighbor (ANN) search algorithm des - **Description**: Distance metric for similarity computation - **Recommendations**: - **L2**: For Euclidean distance (most common) - - **INNER_PRODUCT**: For dot product similarity (use with normalized vectors) + - **INNER_PRODUCT**: For dot product similarity - **COSINE**: For cosine similarity -#### `vector.normalize` -- **Default**: false -- **Description**: Whether to L2-normalize vectors before indexing/searching -- **Recommendations**: - - **true**: When using COSINE metric or when vectors have varying magnitudes - - **false**: When vectors are already normalized or using L2 metric - ### 4. Index Organization #### `vector.size-per-index` diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java index ae0fd9a536f1..e845a93789e5 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java @@ -35,24 +35,12 @@ public class DiskAnnIndex implements Closeable { private final Index index; private final int dimension; - private final DiskAnnVectorMetric metric; - private final DiskAnnIndexType indexType; - private final int maxDegree; private final int buildListSize; private volatile boolean closed = false; - private DiskAnnIndex( - Index index, - int dimension, - DiskAnnVectorMetric metric, - DiskAnnIndexType indexType, - int maxDegree, - int buildListSize) { + private DiskAnnIndex(Index index, int dimension, int buildListSize) { this.index = index; this.dimension = dimension; - this.metric = metric; - this.indexType = indexType; - this.maxDegree = maxDegree; this.buildListSize = buildListSize; } @@ -65,7 +53,7 @@ public static DiskAnnIndex create( MetricType metricType = metric.toMetricType(); Index index = Index.create(dimension, metricType, indexType.value(), maxDegree, buildListSize); - return new DiskAnnIndex(index, dimension, metric, indexType, maxDegree, buildListSize); + return new DiskAnnIndex(index, dimension, buildListSize); } public void addWithIds(ByteBuffer vectorBuffer, ByteBuffer idBuffer, int n) { @@ -85,62 +73,22 @@ public void build() { index.build(buildListSize); } - public void search( - float[] queryVectors, - int n, - int k, - int searchListSize, - float[] distances, - long[] labels) { - ensureOpen(); - if (queryVectors.length < n * dimension) { - throw new IllegalArgumentException( - "Query vectors array too small: required " - + (n * dimension) - + ", got " - + queryVectors.length); - } - if (distances.length < n * k) { - throw new IllegalArgumentException( - "Distances array too small: required " + (n * k) + ", got " + distances.length); - } - if (labels.length < n * k) { - throw new IllegalArgumentException( - "Labels array too small: required " + (n * k) + ", got " + labels.length); - } - index.search(n, queryVectors, k, searchListSize, distances, labels); - } - - public long size() { - ensureOpen(); - return index.getCount(); - } - - public int dimension() { - return dimension; - } - - public DiskAnnVectorMetric metric() { - return metric; - } - - public DiskAnnIndexType indexType() { - return indexType; - } - - public int maxDegree() { - return maxDegree; - } - - public int buildListSize() { - return buildListSize; - } - + /** Return the number of bytes needed for serialization. */ public long serializeSize() { ensureOpen(); return index.serializeSize(); } + /** + * Serialize this index with its Vamana graph adjacency lists into the given direct ByteBuffer. + * + *

The serialized data is later split into an index file (header + graph) and a data file + * (raw vectors) by the writer, then loaded by {@link DiskAnnVectorGlobalIndexReader} for + * search. + * + * @param buffer a direct ByteBuffer of at least {@link #serializeSize()} bytes + * @return the number of bytes written + */ public long serialize(ByteBuffer buffer) { ensureOpen(); if (!buffer.isDirect()) { @@ -149,27 +97,6 @@ public long serialize(ByteBuffer buffer) { return index.serialize(buffer); } - public static DiskAnnIndex deserialize(byte[] data, DiskAnnVectorMetric metric) { - Index index = Index.deserialize(data); - return new DiskAnnIndex( - index, index.getDimension(), metric, DiskAnnIndexType.UNKNOWN, 64, 100); - } - - /** - * Reset the index (remove all vectors). - * - *

Note: This is not supported in the current implementation. DiskANN indices are immutable - * once built. To "reset", you must create a new index. - * - * @throws UnsupportedOperationException always, as reset is not currently supported - */ - public void reset() { - throw new UnsupportedOperationException( - "Reset is not supported for DiskANN indices. " - + "DiskANN indices are immutable once built. " - + "Please create a new index instead."); - } - public static ByteBuffer allocateVectorBuffer(int numVectors, int dimension) { return ByteBuffer.allocateDirect(numVectors * dimension * Float.BYTES) .order(ByteOrder.nativeOrder()); diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexMeta.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexMeta.java index aa3af7738a65..d603250d1ed3 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexMeta.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexMeta.java @@ -25,7 +25,17 @@ import java.io.IOException; import java.io.Serializable; -/** Metadata for DiskANN vector index. */ +/** + * Metadata for DiskANN vector index. + * + *

Stores the file names of the companion files that live alongside the index file: + * + *

    + *
  • {@link #dataFileName()} — raw vector data file + *
  • {@link #pqPivotsFileName()} — PQ codebook (pivots) + *
  • {@link #pqCompressedFileName()} — PQ compressed codes + *
+ */ public class DiskAnnIndexMeta implements Serializable { private static final long serialVersionUID = 1L; @@ -38,31 +48,29 @@ public class DiskAnnIndexMeta implements Serializable { private final long numVectors; private final long minId; private final long maxId; + private final String dataFileName; + private final String pqPivotsFileName; + private final String pqCompressedFileName; public DiskAnnIndexMeta( - int dim, int metricValue, int indexTypeValue, long numVectors, long minId, long maxId) { + int dim, + int metricValue, + int indexTypeValue, + long numVectors, + long minId, + long maxId, + String dataFileName, + String pqPivotsFileName, + String pqCompressedFileName) { this.dim = dim; this.metricValue = metricValue; this.indexTypeValue = indexTypeValue; this.numVectors = numVectors; this.minId = minId; this.maxId = maxId; - } - - public int dim() { - return dim; - } - - public int metricValue() { - return metricValue; - } - - public int indexTypeValue() { - return indexTypeValue; - } - - public long numVectors() { - return numVectors; + this.dataFileName = dataFileName; + this.pqPivotsFileName = pqPivotsFileName; + this.pqCompressedFileName = pqCompressedFileName; } public long minId() { @@ -73,6 +81,11 @@ public long maxId() { return maxId; } + /** The file name of the separate vector data file. */ + public String dataFileName() { + return dataFileName; + } + /** Serialize metadata to byte array. */ public byte[] serialize() throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); @@ -84,6 +97,9 @@ public byte[] serialize() throws IOException { out.writeLong(numVectors); out.writeLong(minId); out.writeLong(maxId); + out.writeUTF(dataFileName); + out.writeUTF(pqPivotsFileName); + out.writeUTF(pqCompressedFileName); out.flush(); return baos.toByteArray(); } @@ -101,6 +117,18 @@ public static DiskAnnIndexMeta deserialize(byte[] data) throws IOException { long numVectors = in.readLong(); long minId = in.readLong(); long maxId = in.readLong(); - return new DiskAnnIndexMeta(dim, metricValue, indexTypeValue, numVectors, minId, maxId); + String dataFileName = in.readUTF(); + String pqPivotsFileName = in.readUTF(); + String pqCompressedFileName = in.readUTF(); + return new DiskAnnIndexMeta( + dim, + metricValue, + indexTypeValue, + numVectors, + minId, + maxId, + dataFileName, + pqPivotsFileName, + pqCompressedFileName); } } diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexType.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexType.java index 101330164777..36612278e08f 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexType.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexType.java @@ -20,9 +20,7 @@ /** DiskANN index type. */ public enum DiskAnnIndexType { - MEMORY(0), - DISK(1), - UNKNOWN(-1); + MEMORY(0); private final int value; diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java index dac7db526016..9a89bdfe9317 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java @@ -18,6 +18,8 @@ package org.apache.paimon.diskann.index; +import org.apache.paimon.diskann.IndexSearcher; +import org.apache.paimon.fs.Path; import org.apache.paimon.fs.SeekableInputStream; import org.apache.paimon.globalindex.GlobalIndexIOMeta; import org.apache.paimon.globalindex.GlobalIndexReader; @@ -31,17 +33,14 @@ import org.apache.paimon.utils.IOUtils; import org.apache.paimon.utils.RoaringNavigableMap64; -import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; -import java.nio.file.Files; import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.PriorityQueue; -import java.util.UUID; /** * Vector global index reader using DiskANN. @@ -52,9 +51,13 @@ */ public class DiskAnnVectorGlobalIndexReader implements GlobalIndexReader { - private final List indices; + /** + * Loaded search handles. Each entry wraps a DiskANN {@link IndexSearcher} (Rust native beam + * search with graph in memory, full vectors read on-demand via {@link FileIOVectorReader}). + */ + private final List handles; + private final List indexMetas; - private final List localIndexFiles; private final List ioMetas; private final GlobalIndexFileReader fileReader; private final DataType fieldType; @@ -62,6 +65,11 @@ public class DiskAnnVectorGlobalIndexReader implements GlobalIndexReader { private volatile boolean metasLoaded = false; private volatile boolean indicesLoaded = false; + /** + * Number of vectors to cache per searcher in the LRU cache inside {@link FileIOVectorReader}. + */ + private static final int VECTOR_CACHE_SIZE = 4096; + public DiskAnnVectorGlobalIndexReader( GlobalIndexFileReader fileReader, List ioMetas, @@ -71,9 +79,50 @@ public DiskAnnVectorGlobalIndexReader( this.ioMetas = ioMetas; this.fieldType = fieldType; this.options = options; - this.indices = new ArrayList<>(); + this.handles = new ArrayList<>(); this.indexMetas = new ArrayList<>(); - this.localIndexFiles = new ArrayList<>(); + } + + /** Wrapper around a search implementation for lifecycle management. */ + private interface SearchHandle extends AutoCloseable { + void search( + long n, + float[] queryVectors, + int k, + int searchListSize, + float[] distances, + long[] labels); + + @Override + void close() throws IOException; + } + + /** + * Uses DiskANN's native Rust beam search via {@link IndexSearcher}. Graph is in memory; vectors + * are fetched on demand from object storage through {@link FileIOVectorReader} JNI callbacks. + */ + private static class DiskAnnSearchHandle implements SearchHandle { + private final IndexSearcher searcher; + + DiskAnnSearchHandle(IndexSearcher searcher) { + this.searcher = searcher; + } + + @Override + public void search( + long n, + float[] queryVectors, + int k, + int searchListSize, + float[] distances, + long[] labels) { + searcher.search(n, queryVectors, k, searchListSize, distances, labels); + } + + @Override + public void close() { + searcher.close(); + } } @Override @@ -123,9 +172,6 @@ private boolean hasOverlap(long minId, long maxId, RoaringNavigableMap64 include private GlobalIndexResult search(VectorSearch vectorSearch) throws IOException { validateVectorType(vectorSearch.vector()); float[] queryVector = ((float[]) vectorSearch.vector()).clone(); - if (options.normalize()) { - normalizeL2(queryVector); - } int limit = vectorSearch.limit(); PriorityQueue result = @@ -140,11 +186,11 @@ private GlobalIndexResult search(VectorSearch vectorSearch) throws IOException { (int) includeRowIds.getLongCardinality()); } - for (DiskAnnIndex index : indices) { - if (index == null) { + for (SearchHandle handle : handles) { + if (handle == null) { continue; } - int effectiveK = (int) Math.min(searchK, index.size()); + int effectiveK = searchK; if (effectiveK <= 0) { continue; } @@ -155,7 +201,7 @@ private GlobalIndexResult search(VectorSearch vectorSearch) throws IOException { // Dynamic search list sizing: use max of configured value and effectiveK // This follows Milvus best practice: search_list should be >= topk int dynamicSearchListSize = Math.max(options.searchListSize(), effectiveK); - index.search(queryVector, 1, effectiveK, dynamicSearchListSize, distances, labels); + handle.search(1, queryVector, effectiveK, dynamicSearchListSize, distances, labels); for (int i = 0; i < effectiveK; i++) { long rowId = labels[i]; @@ -239,99 +285,93 @@ private void ensureLoadAllIndices() throws IOException { private void ensureLoadIndices(List positions) throws IOException { synchronized (this) { - while (indices.size() < ioMetas.size()) { - indices.add(null); + while (handles.size() < ioMetas.size()) { + handles.add(null); } for (int pos : positions) { - if (indices.get(pos) == null) { + if (handles.get(pos) == null) { loadIndexAt(pos); } } } } + /** + * Load an index at the given position. + * + *

The index file (header + graph) and the data file (vectors) are accessed on demand via + * {@link SeekableInputStream}s — neither is loaded into Java memory in full. The PQ pivots and + * compressed codes are loaded into memory as the "memory thumbnail" for approximate distance + * computation during native beam search. + */ private void loadIndexAt(int position) throws IOException { GlobalIndexIOMeta ioMeta = ioMetas.get(position); - DiskAnnIndex index = null; - try (SeekableInputStream in = fileReader.getInputStream(ioMeta)) { - index = loadIndex(in); - if (indices.size() <= position) { - while (indices.size() < position) { - indices.add(null); + DiskAnnIndexMeta meta = indexMetas.get(position); + SearchHandle handle = null; + try { + // 1. Open index file (header + graph) as a SeekableInputStream. + // FileIOGraphReader scans the header + builds offset index; graph neighbors are read + // on demand during beam search. + SeekableInputStream graphStream = fileReader.getInputStream(ioMeta); + FileIOGraphReader graphReader = new FileIOGraphReader(graphStream, VECTOR_CACHE_SIZE); + + // 2. Open data file stream for on-demand full-vector reads. + Path dataPath = new Path(ioMeta.filePath().getParent(), meta.dataFileName()); + GlobalIndexIOMeta dataIOMeta = new GlobalIndexIOMeta(dataPath, 0L, new byte[0]); + SeekableInputStream vectorStream = fileReader.getInputStream(dataIOMeta); + FileIOVectorReader vectorReader = + new FileIOVectorReader( + vectorStream, + graphReader.getDimension(), + 0L, // vectors start at offset 0 in the data file + buildExtIdToEntryIndex(graphReader), + VECTOR_CACHE_SIZE); + + // 3. Create DiskANN native searcher with on-demand graph + vector access. + handle = + new DiskAnnSearchHandle( + IndexSearcher.createFromReaders( + graphReader, vectorReader, graphReader.getDimension())); + + if (handles.size() <= position) { + while (handles.size() < position) { + handles.add(null); } - indices.add(index); + handles.add(handle); } else { - indices.set(position, index); + handles.set(position, handle); } } catch (Exception e) { - IOUtils.closeQuietly(index); - throw e; - } - } - - private DiskAnnIndex loadIndex(SeekableInputStream in) throws IOException { - // For better memory efficiency, write to a temporary file - // This allows the OS to manage memory more efficiently for large indices - File tempIndexFile = - Files.createTempFile("paimon-diskann-" + UUID.randomUUID(), ".index").toFile(); - localIndexFiles.add(tempIndexFile); - - // Copy index data to temp file - try (FileOutputStream fos = new FileOutputStream(tempIndexFile)) { - byte[] buffer = new byte[32768]; - int bytesRead; - while ((bytesRead = in.read(buffer)) != -1) { - fos.write(buffer, 0, bytesRead); - } + IOUtils.closeQuietly(handle); + throw e instanceof IOException ? (IOException) e : new IOException(e); } - - // Load from file for potential mmap benefits - // Note: Current implementation still deserializes to memory - // Future enhancement: Add native file-based loading if supported - byte[] data = Files.readAllBytes(tempIndexFile.toPath()); - return DiskAnnIndex.deserialize(data, options.metric()); } - private void normalizeL2(float[] vector) { - float norm = 0.0f; - for (float v : vector) { - norm += v * v; - } - norm = (float) Math.sqrt(norm); - if (norm > 0) { - for (int i = 0; i < vector.length; i++) { - vector[i] /= norm; - } + /** + * Build the extId → entryIndex mapping from the graph reader's ID arrays. This is needed by + * {@link FileIOVectorReader} to map external vector IDs to byte offsets in the data file. + */ + private static Map buildExtIdToEntryIndex(FileIOGraphReader graphReader) { + int[] intIds = graphReader.getAllInternalIds(); + long[] extIds = graphReader.getAllExternalIds(); + Map map = new HashMap<>(intIds.length); + for (int i = 0; i < intIds.length; i++) { + map.put(extIds[i], i); } + return map; } @Override public void close() throws IOException { Throwable firstException = null; - // Close all DiskANN indices - for (DiskAnnIndex index : indices) { - if (index == null) { + // Close all search handles (also closes their FileIOVectorReader streams). + for (SearchHandle handle : handles) { + if (handle == null) { continue; } try { - index.close(); - } catch (Throwable t) { - if (firstException == null) { - firstException = t; - } else { - firstException.addSuppressed(t); - } - } - } - indices.clear(); - - // Delete temporary files - for (File tempFile : localIndexFiles) { - try { - if (tempFile != null && tempFile.exists()) { - tempFile.delete(); - } + handle.close(); } catch (Throwable t) { if (firstException == null) { firstException = t; @@ -340,7 +380,7 @@ public void close() throws IOException { } } } - localIndexFiles.clear(); + handles.clear(); if (firstException != null) { if (firstException instanceof IOException) { diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java index ca60a59babc9..a4d0a56459ef 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java @@ -40,8 +40,26 @@ /** * Vector global index writer using DiskANN. * - *

Vectors are added to the index in batches. When the current index reaches {@code sizePerIndex} - * vectors, it is built and serialized to a file and a new index is created. + *

The build pipeline follows two phases: + * + *

    + *
  1. Vamana Graph Construction — vectors are added in batches and the Vamana graph (with + * alpha-pruning) is built via the native DiskANN library. + *
  2. PQ Compression — after the graph is built, a Product Quantization codebook is + * trained on the vectors and all vectors are compressed to compact PQ codes. + *
+ * + *

For each index flush, four files are produced: + * + *

    + *
  • {@code .index} — Vamana graph (header + adjacency lists) + *
  • {@code .data} — raw vector data (for exact distance reranking) + *
  • {@code .pq_pivots} — PQ codebook (centroids) + *
  • {@code .pq_compressed} — PQ compressed codes (memory thumbnail) + *
+ * + *

The PQ compressed data acts as a "memory thumbnail" — during search it stays resident in + * memory and allows fast approximate distance computation, reducing disk I/O for full vectors. * *

This class implements {@link Closeable} so that the native DiskANN index is released even if * {@link #finish()} is never called or throws an exception. @@ -66,6 +84,9 @@ public class DiskAnnVectorGlobalIndexWriter implements GlobalIndexSingletonWrite private DiskAnnIndex currentIndex; private boolean built = false; + /** All vectors accumulated for the current index, kept for PQ training after graph build. */ + private final List currentIndexVectors; + public DiskAnnVectorGlobalIndexWriter( GlobalIndexFileWriter fileWriter, DataType fieldType, @@ -78,6 +99,7 @@ public DiskAnnVectorGlobalIndexWriter( this.dim = options.dimension(); this.pendingBatch = new ArrayList<>(batchSize); this.results = new ArrayList<>(); + this.currentIndexVectors = new ArrayList<>(); validateFieldType(fieldType); } @@ -106,9 +128,6 @@ public void write(Object fieldData) { "Unsupported vector type: " + fieldData.getClass().getName()); } checkDimension(vector); - if (options.normalize()) { - normalizeL2(vector); - } currentIndexMinId = Math.min(currentIndexMinId, count); currentIndexMaxId = Math.max(currentIndexMaxId, count); pendingBatch.add(new VectorEntry(count, vector)); @@ -164,6 +183,8 @@ private void addBatchToIndex() throws IOException { floatView.put(i * dim + j, vector[j]); } longView.put(i, entry.id); + // Accumulate vectors for PQ training. + currentIndexVectors.add(vector); } currentIndex.addWithIds(vectorBuffer, idBuffer, n); @@ -177,31 +198,82 @@ private void flushCurrentIndex() throws IOException { return; } + // ---- Phase 2: Vamana graph construction ---- if (!built) { currentIndex.build(); built = true; } - String fileName = fileWriter.newFileName(DiskAnnVectorGlobalIndexerFactory.IDENTIFIER); - try (OutputStream out = new BufferedOutputStream(fileWriter.newOutputStream(fileName))) { - long serializeSize = currentIndex.serializeSize(); - if (serializeSize > Integer.MAX_VALUE) { - throw new IOException( - "Index too large to serialize: " - + serializeSize - + " bytes exceeds maximum buffer size"); - } - ByteBuffer serializeBuffer = - ByteBuffer.allocateDirect((int) serializeSize).order(ByteOrder.nativeOrder()); - long bytesWritten = currentIndex.serialize(serializeBuffer); - - byte[] indexData = new byte[(int) bytesWritten]; - serializeBuffer.rewind(); - serializeBuffer.get(indexData); - out.write(indexData); + // Serialize the full index (header + graph + vectors) into one buffer. + long serializeSize = currentIndex.serializeSize(); + if (serializeSize > Integer.MAX_VALUE) { + throw new IOException( + "Index too large to serialize: " + + serializeSize + + " bytes exceeds maximum buffer size"); + } + + ByteBuffer serializeBuffer = + ByteBuffer.allocateDirect((int) serializeSize).order(ByteOrder.nativeOrder()); + long bytesWritten = currentIndex.serialize(serializeBuffer); + + byte[] fullData = new byte[(int) bytesWritten]; + serializeBuffer.rewind(); + serializeBuffer.get(fullData); + + // Parse layout to find the boundary between graph and vector sections. + FileIOVectorReader.IndexLayout layout = FileIOVectorReader.parseIndexLayout(fullData); + int vectorOffset = (int) layout.vectorSectionOffset(); + + // Generate file names — all share the same base name. + String indexFileName = fileWriter.newFileName(DiskAnnVectorGlobalIndexerFactory.IDENTIFIER); + String baseName = indexFileName.replaceAll("\\.index$", ""); + String dataFileName = baseName + ".data"; + String pqPivotsFileName = baseName + ".pq_pivots"; + String pqCompressedFileName = baseName + ".pq_compressed"; + + // Write index file: header + graph section only. + try (OutputStream out = + new BufferedOutputStream(fileWriter.newOutputStream(indexFileName))) { + out.write(fullData, 0, vectorOffset); + out.flush(); + } + + // Write data file: raw vector section only. + try (OutputStream out = + new BufferedOutputStream(fileWriter.newOutputStream(dataFileName))) { + out.write(fullData, vectorOffset, fullData.length - vectorOffset); + out.flush(); + } + + // ---- Phase 1: PQ Compression & Training ---- + // Train PQ codebook on the accumulated vectors and compress them. + float[][] vectors = currentIndexVectors.toArray(new float[0][]); + ProductQuantizer pq = + ProductQuantizer.train( + vectors, + dim, + options.pqSubspaces(), + options.pqSampleSize(), + options.pqKmeansIterations()); + + byte[][] codes = pq.encodeAll(vectors); + + // Write PQ pivots file (codebook). + try (OutputStream out = + new BufferedOutputStream(fileWriter.newOutputStream(pqPivotsFileName))) { + out.write(pq.serializePivots()); out.flush(); } + // Write PQ compressed file (memory thumbnail). + try (OutputStream out = + new BufferedOutputStream(fileWriter.newOutputStream(pqCompressedFileName))) { + out.write(ProductQuantizer.serializeCompressed(codes, options.pqSubspaces())); + out.flush(); + } + + // Build metadata with all companion file names. DiskAnnIndexMeta meta = new DiskAnnIndexMeta( dim, @@ -209,14 +281,18 @@ private void flushCurrentIndex() throws IOException { options.indexType().value(), currentIndexCount, currentIndexMinId, - currentIndexMaxId); - results.add(new ResultEntry(fileName, currentIndexCount, meta.serialize())); + currentIndexMaxId, + dataFileName, + pqPivotsFileName, + pqCompressedFileName); + results.add(new ResultEntry(indexFileName, currentIndexCount, meta.serialize())); currentIndex.close(); currentIndex = null; currentIndexCount = 0; currentIndexMinId = Long.MAX_VALUE; currentIndexMaxId = Long.MIN_VALUE; + currentIndexVectors.clear(); built = false; } @@ -238,19 +314,6 @@ private void checkDimension(float[] vector) { } } - private void normalizeL2(float[] vector) { - float norm = 0.0f; - for (float v : vector) { - norm += v * v; - } - norm = (float) Math.sqrt(norm); - if (norm > 0) { - for (int i = 0; i < vector.length; i++) { - vector[i] /= norm; - } - } - } - /** * Release native resources held by the current in-progress index. * @@ -264,6 +327,7 @@ public void close() { currentIndex.close(); currentIndex = null; } + currentIndexVectors.clear(); } /** Entry holding a vector and its row ID. */ diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorIndexOptions.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorIndexOptions.java index e7da850cd1de..8dab2a44784f 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorIndexOptions.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorIndexOptions.java @@ -76,13 +76,26 @@ public class DiskAnnVectorIndexOptions { "The multiplier for the search limit when filtering is applied. " + "This is used to fetch more results to ensure enough records after filtering."); - public static final ConfigOption VECTOR_NORMALIZE = - ConfigOptions.key("vector.normalize") - .booleanType() - .defaultValue(false) + public static final ConfigOption VECTOR_PQ_SUBSPACES = + ConfigOptions.key("vector.diskann.pq-subspaces") + .intType() + .defaultValue(-1) .withDescription( - "Whether to L2 normalize vectors before indexing and searching. " - + "Enabling this converts L2 distance to cosine similarity."); + "Number of subspaces (M) for Product Quantization. " + + "Dimension must be divisible by M. " + + "Default (-1) auto-computes as max(1, dim/4)."); + + public static final ConfigOption VECTOR_PQ_KMEANS_ITERATIONS = + ConfigOptions.key("vector.diskann.pq-kmeans-iterations") + .intType() + .defaultValue(20) + .withDescription("Number of K-Means iterations for PQ codebook training."); + + public static final ConfigOption VECTOR_PQ_SAMPLE_SIZE = + ConfigOptions.key("vector.diskann.pq-sample-size") + .intType() + .defaultValue(100_000) + .withDescription("Maximum number of vectors sampled for PQ codebook training."); private final int dimension; private final DiskAnnVectorMetric metric; @@ -92,7 +105,9 @@ public class DiskAnnVectorIndexOptions { private final int searchListSize; private final int sizePerIndex; private final int searchFactor; - private final boolean normalize; + private final int pqSubspaces; + private final int pqKmeansIterations; + private final int pqSampleSize; public DiskAnnVectorIndexOptions(Options options) { this.dimension = options.get(VECTOR_DIM); @@ -106,7 +121,12 @@ public DiskAnnVectorIndexOptions(Options options) { ? options.get(VECTOR_SIZE_PER_INDEX) : VECTOR_SIZE_PER_INDEX.defaultValue(); this.searchFactor = options.get(VECTOR_SEARCH_FACTOR); - this.normalize = options.get(VECTOR_NORMALIZE); + + int rawPqSub = options.get(VECTOR_PQ_SUBSPACES); + this.pqSubspaces = + rawPqSub > 0 ? rawPqSub : ProductQuantizer.defaultNumSubspaces(dimension); + this.pqKmeansIterations = options.get(VECTOR_PQ_KMEANS_ITERATIONS); + this.pqSampleSize = options.get(VECTOR_PQ_SAMPLE_SIZE); } public int dimension() { @@ -141,7 +161,18 @@ public int searchFactor() { return searchFactor; } - public boolean normalize() { - return normalize; + /** Number of PQ subspaces (M). */ + public int pqSubspaces() { + return pqSubspaces; + } + + /** Number of K-Means iterations for PQ training. */ + public int pqKmeansIterations() { + return pqKmeansIterations; + } + + /** Maximum number of training samples for PQ. */ + public int pqSampleSize() { + return pqSampleSize; } } diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOGraphReader.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOGraphReader.java new file mode 100644 index 000000000000..c366e184b106 --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOGraphReader.java @@ -0,0 +1,293 @@ +/* + * 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.paimon.diskann.index; + +import org.apache.paimon.fs.SeekableInputStream; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * Reads graph structure (neighbors) from a DiskANN index file on demand via a Paimon {@link + * SeekableInputStream}. + * + *

The underlying stream can be backed by any Paimon FileIO provider — local, HDFS, S3, OSS, etc. + * This class adds an LRU cache so that repeated reads for the same node (common during DiskANN's + * beam search) do not trigger redundant I/O. + * + *

The Rust JNI layer invokes {@link #readNeighbors(int)} via reflection during DiskANN's native + * beam search. It also calls getter methods ({@link #getDimension()}, {@link #getCount()}, etc.) + * during searcher initialization. + * + *

Index file layout

+ * + *
+ *   Header (36 bytes): 9 × i32
+ *     magic, version, dimension, metricType, indexType,
+ *     maxDegree, buildListSize, count, startId
+ *
+ *   Graph section: for each node (count nodes):
+ *     ext_id       : i64
+ *     int_id       : i32
+ *     neighbor_cnt : i32
+ *     neighbors    : neighbor_cnt × i32
+ * 
+ * + *

On construction, the header is read and the graph section is scanned once sequentially to + * build an offset index (mapping internal node ID → file byte offset for its neighbor data). After + * that, individual neighbor lists are read on demand by seeking to the stored offset. + */ +public class FileIOGraphReader implements Closeable { + + /** Header size: 9 × i32 = 36 bytes. */ + private static final int HEADER_SIZE = 36; + + /** Source stream — must support seek(). */ + private final SeekableInputStream input; + + // ---- Header fields ---- + private final int dimension; + private final int metricValue; + private final int indexTypeValue; + private final int maxDegree; + private final int buildListSize; + private final int count; + private final int startId; + + // ---- Offset index built during initial scan ---- + + /** Mapping from internal node ID → byte offset of the node's neighbor_cnt field in the file. */ + private final Map nodeNeighborOffsets; + + /** All internal node IDs (in file order). */ + private final int[] allInternalIds; + + /** Corresponding external IDs (same order as {@link #allInternalIds}). */ + private final long[] allExternalIds; + + /** LRU cache: internal node ID → neighbor list (int[]). */ + private final LinkedHashMap cache; + + /** + * Create a reader by parsing the header and scanning the graph section to build the offset + * index. + * + * @param input seekable input stream for the index file (header + graph) + * @param cacheSize maximum number of cached neighbor lists (0 uses default 4096) + * @throws IOException if reading or parsing fails + */ + public FileIOGraphReader(SeekableInputStream input, int cacheSize) throws IOException { + this.input = input; + + // 1. Read header. + byte[] headerBuf = new byte[HEADER_SIZE]; + input.seek(0); + readFully(input, headerBuf); + + int off = 0; + // magic(4) + version(4) — skip validation here; Rust validates during search. + off += 8; + this.dimension = readInt(headerBuf, off); + off += 4; + this.metricValue = readInt(headerBuf, off); + off += 4; + this.indexTypeValue = readInt(headerBuf, off); + off += 4; + this.maxDegree = readInt(headerBuf, off); + off += 4; + this.buildListSize = readInt(headerBuf, off); + off += 4; + this.count = readInt(headerBuf, off); + off += 4; + this.startId = readInt(headerBuf, off); + + // 2. Scan graph section to build offset index. + this.nodeNeighborOffsets = new HashMap<>(count); + this.allInternalIds = new int[count]; + this.allExternalIds = new long[count]; + + // Reusable buffer for reading ext_id(8) + int_id(4) + neighbor_cnt(4) = 16 bytes per node. + byte[] nodeBuf = new byte[16]; + long filePos = HEADER_SIZE; + + for (int i = 0; i < count; i++) { + input.seek(filePos); + readFully(input, nodeBuf); + + long extId = readLong(nodeBuf, 0); + int intId = readInt(nodeBuf, 8); + int neighborCount = readInt(nodeBuf, 12); + + allInternalIds[i] = intId; + allExternalIds[i] = extId; + + // Store file offset pointing to the neighbor_cnt field (so readNeighbors can re-read + // count + data). + nodeNeighborOffsets.put(intId, filePos + 12); + + // Advance past: ext_id(8) + int_id(4) + neighbor_cnt(4) + neighbors(cnt*4). + filePos += 16 + (long) neighborCount * 4; + } + + // 3. Create LRU cache. + final int cap = cacheSize > 0 ? cacheSize : 4096; + this.cache = + new LinkedHashMap(cap, 0.75f, true) { + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + return size() > cap; + } + }; + } + + // ---- Header accessors (called by Rust JNI during initialization) ---- + + public int getDimension() { + return dimension; + } + + public int getMetricValue() { + return metricValue; + } + + public int getIndexTypeValue() { + return indexTypeValue; + } + + public int getMaxDegree() { + return maxDegree; + } + + public int getBuildListSize() { + return buildListSize; + } + + public int getCount() { + return count; + } + + public int getStartId() { + return startId; + } + + /** Return all internal node IDs (in file order). */ + public int[] getAllInternalIds() { + return allInternalIds; + } + + /** Return all external node IDs (same order as {@link #getAllInternalIds()}). */ + public long[] getAllExternalIds() { + return allExternalIds; + } + + // ---- On-demand neighbor reading (called by Rust JNI during beam search) ---- + + /** + * Read the neighbor list of the given internal node. + * + *

Called by the Rust JNI layer during DiskANN's native beam search. Returns a defensive + * copy — callers may freely modify the returned array without corrupting the cache. + * + * @param internalNodeId the internal (graph) node ID + * @return the neighbor internal IDs, or an empty array if the node is unknown + */ + public int[] readNeighbors(int internalNodeId) { + // 1. LRU cache hit — return a defensive copy. + int[] cached = cache.get(internalNodeId); + if (cached != null) { + return Arrays.copyOf(cached, cached.length); + } + + // 2. Look up file offset. + Long offset = nodeNeighborOffsets.get(internalNodeId); + if (offset == null) { + return new int[0]; // unknown node + } + + // 3. Seek & read neighbor_cnt + neighbor_ids. + try { + input.seek(offset); + + // Read neighbor count (4 bytes). + byte[] cntBuf = new byte[4]; + readFully(input, cntBuf); + int neighborCount = readInt(cntBuf, 0); + + // Read neighbor IDs. + byte[] neighborBuf = new byte[neighborCount * 4]; + readFully(input, neighborBuf); + + int[] neighbors = new int[neighborCount]; + ByteBuffer bb = ByteBuffer.wrap(neighborBuf).order(ByteOrder.nativeOrder()); + bb.asIntBuffer().get(neighbors); + + // 4. Cache a copy. + cache.put(internalNodeId, Arrays.copyOf(neighbors, neighbors.length)); + return neighbors; + } catch (IOException e) { + throw new RuntimeException( + "Failed to read neighbors for node " + internalNodeId + " at offset " + offset, + e); + } + } + + @Override + public void close() throws IOException { + cache.clear(); + input.close(); + } + + // ---- Helpers ---- + + private static void readFully(SeekableInputStream in, byte[] buf) throws IOException { + int off = 0; + while (off < buf.length) { + int n = in.read(buf, off, buf.length - off); + if (n < 0) { + throw new IOException( + "Unexpected end of stream at offset " + off + " of " + buf.length); + } + off += n; + } + } + + private static int readInt(byte[] buf, int off) { + return (buf[off] & 0xFF) + | ((buf[off + 1] & 0xFF) << 8) + | ((buf[off + 2] & 0xFF) << 16) + | ((buf[off + 3] & 0xFF) << 24); + } + + private static long readLong(byte[] buf, int off) { + return (buf[off] & 0xFFL) + | ((buf[off + 1] & 0xFFL) << 8) + | ((buf[off + 2] & 0xFFL) << 16) + | ((buf[off + 3] & 0xFFL) << 24) + | ((buf[off + 4] & 0xFFL) << 32) + | ((buf[off + 5] & 0xFFL) << 40) + | ((buf[off + 6] & 0xFFL) << 48) + | ((buf[off + 7] & 0xFFL) << 56); + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOVectorReader.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOVectorReader.java new file mode 100644 index 000000000000..528724cdfb80 --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOVectorReader.java @@ -0,0 +1,223 @@ +/* + * 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.paimon.diskann.index; + +import org.apache.paimon.fs.SeekableInputStream; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * Fetches vectors from a DiskANN data file through a Paimon {@link SeekableInputStream}. + * + *

The underlying stream can be backed by any Paimon FileIO provider — local, HDFS, S3, OSS (via + * Jindo SDK), etc. This class adds an LRU cache so that repeated reads for the same vector (common + * during DiskANN's beam search) do not trigger redundant I/O. + * + *

The Rust JNI layer invokes {@link #readVector(long)} via reflection during DiskANN's native + * beam search — no specific Java interface is required. + * + *

File layout (vector section)

+ * + *

Vectors are stored contiguously. Each vector occupies {@code dimension * 4} bytes + * (native-order floats). The position of a vector with entry-index {@code i} is: {@code + * vectorSectionOffset + i * dimension * 4}. + */ +public class FileIOVectorReader implements Closeable { + + /** Source stream — must support seek(). */ + private final SeekableInputStream input; + + /** Vector dimension. */ + private final int dimension; + + /** Byte offset of the vector section within the file. Vectors start after the graph section. */ + private final long vectorSectionOffset; + + /** + * Mapping from external (user-facing) vector ID to entry index in the file. Entry index + * determines the byte offset: {@code vectorSectionOffset + entryIndex * dimension * 4}. + */ + private final Map extIdToEntryIndex; + + /** LRU cache: external vector ID → float[]. */ + private final LinkedHashMap cache; + + /** Reusable byte buffer for reading a single vector. */ + private final byte[] readBuf; + + /** + * Create a reader. + * + * @param input seekable input stream for the data file + * @param dimension vector dimension + * @param vectorSectionOffset byte offset where the vector section starts in the file + * @param extIdToEntryIndex mapping from external vector ID to sequential entry index + * @param cacheSize maximum number of cached vectors (0 disables caching) + */ + public FileIOVectorReader( + SeekableInputStream input, + int dimension, + long vectorSectionOffset, + Map extIdToEntryIndex, + int cacheSize) { + this.input = input; + this.dimension = dimension; + this.vectorSectionOffset = vectorSectionOffset; + this.extIdToEntryIndex = extIdToEntryIndex; + this.readBuf = new byte[dimension * Float.BYTES]; + + final int cap = Math.max(cacheSize, 16); + this.cache = + new LinkedHashMap(cap, 0.75f, true) { + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + return size() > cap; + } + }; + } + + /** + * Read the vector associated with the given external ID. + * + *

Called by the Rust JNI layer during DiskANN's native beam search. Returns a defensive + * copy — callers may freely modify the returned array without corrupting the cache. + * + * @param vectorId the external (user-facing) vector ID + * @return the float vector (a fresh copy), or {@code null} if unavailable + */ + public float[] readVector(long vectorId) { + // 1. LRU cache hit — return a defensive copy. + float[] cached = cache.get(vectorId); + if (cached != null) { + return Arrays.copyOf(cached, cached.length); + } + + // 2. Look up entry index. + Integer entryIndex = extIdToEntryIndex.get(vectorId); + if (entryIndex == null) { + return null; // unknown vector + } + + // 3. Seek & read from the underlying stream. + long byteOffset = vectorSectionOffset + (long) entryIndex * dimension * Float.BYTES; + try { + input.seek(byteOffset); + readFully(input, readBuf); + } catch (IOException e) { + throw new RuntimeException( + "Failed to read vector " + vectorId + " at offset " + byteOffset, e); + } + + // 4. Decode floats. + float[] vector = new float[dimension]; + ByteBuffer bb = ByteBuffer.wrap(readBuf).order(ByteOrder.nativeOrder()); + bb.asFloatBuffer().get(vector); + + // 5. Store a separate copy in the cache so the returned array is independent. + cache.put(vectorId, Arrays.copyOf(vector, vector.length)); + return vector; + } + + @Override + public void close() throws IOException { + cache.clear(); + input.close(); + } + + // ------------------------------------------------------------------ + // Helpers + // ------------------------------------------------------------------ + + private static void readFully(SeekableInputStream in, byte[] buf) throws IOException { + int off = 0; + while (off < buf.length) { + int n = in.read(buf, off, buf.length - off); + if (n < 0) { + throw new IOException( + "Unexpected end of stream at offset " + off + " of " + buf.length); + } + off += n; + } + } + + // ------------------------------------------------------------------ + // Factory helper — parse header to obtain vectorSectionOffset + // and extIdToEntryIndex. + // ------------------------------------------------------------------ + + /** + * Parse a serialized byte array (header + graph section) to extract the mapping and vector + * section offset needed by this reader. + * + *

This is typically called once when opening the index for search. The byte array only needs + * to contain the header and graph section — the vector section is not accessed. + * + * @param data byte array containing at least the header and graph section + * @return a {@link IndexLayout} containing the parsed metadata + */ + public static IndexLayout parseIndexLayout(byte[] data) { + // Header: 9 × i32 = 36 bytes. + int off = 0; + // skip magic(4), version(4), dimension(4), metricType(4), indexType(4), + // maxDegree(4), buildListSize(4) + off += 28; + int count = readInt(data, off); + off += 4; + // skip startId(4) + off += 4; + + // Scan graph section to find the vector section offset. + for (int i = 0; i < count; i++) { + // skip ext_id(8) + int_id(4) + off += 12; + int neighborCount = readInt(data, off); + off += 4; + // skip neighbor IDs + off += neighborCount * 4; + } + + return new IndexLayout(off); + } + + /** Parsed layout metadata. */ + public static class IndexLayout { + private final long vectorSectionOffset; + + IndexLayout(long vectorSectionOffset) { + this.vectorSectionOffset = vectorSectionOffset; + } + + public long vectorSectionOffset() { + return vectorSectionOffset; + } + } + + private static int readInt(byte[] buf, int off) { + return (buf[off] & 0xFF) + | ((buf[off + 1] & 0xFF) << 8) + | ((buf[off + 2] & 0xFF) << 16) + | ((buf[off + 3] & 0xFF) << 24); + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/ProductQuantizer.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/ProductQuantizer.java new file mode 100644 index 000000000000..3a26d0c121e8 --- /dev/null +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/ProductQuantizer.java @@ -0,0 +1,396 @@ +/* + * 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.paimon.diskann.index; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.Random; + +/** + * Product Quantization (PQ) implementation for DiskANN. + * + *

PQ compresses high-dimensional vectors into compact codes by: + * + *

    + *
  1. Splitting each vector into {@code M} sub-vectors (subspaces). + *
  2. Training {@code K} centroids per subspace using K-Means clustering. + *
  3. Encoding each sub-vector as the index of its nearest centroid (1 byte for K=256). + *
+ * + *

This produces a "memory thumbnail" — e.g., a 128-dim float vector (512 bytes) is compressed to + * just 16 bytes (with M=16). During search, the compressed codes stay resident in memory and allow + * fast approximate distance computation, reducing unnecessary disk I/O for full vectors. + * + *

File layout

+ * + *

Pivots file ({@code .pq_pivots}): contains the trained codebook. + * + *

+ *   int: dimension (D)
+ *   int: numSubspaces (M)
+ *   int: numCentroids (K)
+ *   int: subDimension (D/M)
+ *   float[M * K * subDim]: centroid data (row-major)
+ * 
+ * + *

Compressed file ({@code .pq_compressed}): contains the PQ codes. + * + *

+ *   int: numVectors (N)
+ *   int: numSubspaces (M)
+ *   byte[N * M]: PQ codes (row-major, one row per vector)
+ * 
+ */ +public class ProductQuantizer { + + /** Number of centroids per subspace. Fixed at 256 so each code fits in one byte (uint8). */ + public static final int NUM_CENTROIDS = 256; + + /** Default number of K-Means iterations. */ + private static final int DEFAULT_KMEANS_ITERATIONS = 20; + + /** Default maximum number of training samples. */ + private static final int DEFAULT_MAX_TRAINING_SAMPLES = 100_000; + + private final int dimension; + private final int numSubspaces; + private final int subDimension; + + /** + * Codebook: {@code centroids[m][k][d]} is the d-th component of the k-th centroid in the m-th + * subspace. + */ + private final float[][][] centroids; + + private ProductQuantizer(int dimension, int numSubspaces, float[][][] centroids) { + this.dimension = dimension; + this.numSubspaces = numSubspaces; + this.subDimension = dimension / numSubspaces; + this.centroids = centroids; + } + + // ------------------------------------------------------------------ + // Training + // ------------------------------------------------------------------ + + /** + * Train a PQ codebook from the given vectors. + * + *

If there are more vectors than {@code maxTrainingSamples}, a random subset is used for + * training. All K-Means runs use K-Means++ initialization. + * + * @param vectors training data; each entry is a float[dimension] + * @param dimension vector dimension (must be divisible by {@code numSubspaces}) + * @param numSubspaces number of subspaces M + * @param maxTrainingSamples maximum number of samples for K-Means training + * @param kmeansIterations number of Lloyd iterations + * @return a trained {@link ProductQuantizer} + */ + public static ProductQuantizer train( + float[][] vectors, + int dimension, + int numSubspaces, + int maxTrainingSamples, + int kmeansIterations) { + if (dimension % numSubspaces != 0) { + throw new IllegalArgumentException( + "Dimension (" + + dimension + + ") must be divisible by numSubspaces (" + + numSubspaces + + ")"); + } + int subDim = dimension / numSubspaces; + int k = Math.min(NUM_CENTROIDS, vectors.length); + Random rng = new Random(42); + + // Subsample if needed. + float[][] samples = vectors; + if (vectors.length > maxTrainingSamples) { + samples = sample(vectors, maxTrainingSamples, rng); + } + + float[][][] codebook = new float[numSubspaces][k][subDim]; + for (int m = 0; m < numSubspaces; m++) { + // Extract sub-vectors for this subspace. + float[][] subVectors = extractSubspace(samples, m, subDim); + codebook[m] = kMeans(subVectors, k, kmeansIterations, rng); + } + return new ProductQuantizer(dimension, numSubspaces, codebook); + } + + // ------------------------------------------------------------------ + // Encoding + // ------------------------------------------------------------------ + + /** + * Encode a single vector into PQ codes. + * + * @param vector float[dimension] + * @return byte[numSubspaces] — one code per subspace + */ + public byte[] encode(float[] vector) { + byte[] codes = new byte[numSubspaces]; + for (int m = 0; m < numSubspaces; m++) { + int offset = m * subDimension; + float bestDist = Float.MAX_VALUE; + int bestIdx = 0; + for (int k = 0; k < centroids[m].length; k++) { + float dist = l2Squared(vector, offset, centroids[m][k], 0, subDimension); + if (dist < bestDist) { + bestDist = dist; + bestIdx = k; + } + } + codes[m] = (byte) bestIdx; + } + return codes; + } + + /** + * Encode all vectors. + * + * @return byte[numVectors][numSubspaces] + */ + public byte[][] encodeAll(float[][] vectors) { + byte[][] codes = new byte[vectors.length][]; + for (int i = 0; i < vectors.length; i++) { + codes[i] = encode(vectors[i]); + } + return codes; + } + + // ------------------------------------------------------------------ + // Serialization — Pivots + // ------------------------------------------------------------------ + + /** Serialize the codebook (pivots) to a byte array. */ + public byte[] serializePivots() throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos); + out.writeInt(dimension); + out.writeInt(numSubspaces); + out.writeInt(centroids[0].length); // K + out.writeInt(subDimension); + for (int m = 0; m < numSubspaces; m++) { + for (int k = 0; k < centroids[m].length; k++) { + for (int d = 0; d < subDimension; d++) { + out.writeFloat(centroids[m][k][d]); + } + } + } + out.flush(); + return baos.toByteArray(); + } + + // ------------------------------------------------------------------ + // Serialization — Compressed codes + // ------------------------------------------------------------------ + + /** + * Serialize PQ compressed codes to a byte array. + * + * @param codes byte[numVectors][numSubspaces] + * @param numSubspaces M + * @return serialized bytes + */ + public static byte[] serializeCompressed(byte[][] codes, int numSubspaces) throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos); + out.writeInt(codes.length); + out.writeInt(numSubspaces); + for (byte[] code : codes) { + out.write(code); + } + out.flush(); + return baos.toByteArray(); + } + + /** + * Compute a reasonable default number of subspaces for the given dimension. The result is the + * largest divisor of {@code dim} that is {@code <= dim / 4} and at least 1. + */ + public static int defaultNumSubspaces(int dim) { + int target = Math.max(1, dim / 4); + while (target > 1 && dim % target != 0) { + target--; + } + return target; + } + + // ------------------------------------------------------------------ + // K-Means (K-Means++ init + Lloyd iterations) + // ------------------------------------------------------------------ + + /** + * Run K-Means clustering on the given data. + * + * @param data float[n][subDim] + * @param k number of clusters + * @param maxIter maximum iterations + * @param rng random source + * @return float[k][subDim] — cluster centroids + */ + static float[][] kMeans(float[][] data, int k, int maxIter, Random rng) { + int n = data.length; + int d = data[0].length; + + if (n <= k) { + // Fewer data points than clusters — each point is its own centroid. + float[][] centroids = new float[k][d]; + for (int i = 0; i < n; i++) { + centroids[i] = Arrays.copyOf(data[i], d); + } + for (int i = n; i < k; i++) { + centroids[i] = Arrays.copyOf(data[rng.nextInt(n)], d); + } + return centroids; + } + + // K-Means++ initialization. + float[][] centroids = kMeansPPInit(data, k, rng); + + int[] assignments = new int[n]; + int[] counts = new int[k]; + + for (int iter = 0; iter < maxIter; iter++) { + // Assignment step. + for (int i = 0; i < n; i++) { + float bestDist = Float.MAX_VALUE; + int bestC = 0; + for (int c = 0; c < k; c++) { + float dist = l2Squared(data[i], 0, centroids[c], 0, d); + if (dist < bestDist) { + bestDist = dist; + bestC = c; + } + } + assignments[i] = bestC; + } + + // Update step. + float[][] newCentroids = new float[k][d]; + Arrays.fill(counts, 0); + for (int i = 0; i < n; i++) { + int c = assignments[i]; + counts[c]++; + for (int j = 0; j < d; j++) { + newCentroids[c][j] += data[i][j]; + } + } + for (int c = 0; c < k; c++) { + if (counts[c] > 0) { + for (int j = 0; j < d; j++) { + newCentroids[c][j] /= counts[c]; + } + } else { + // Re-seed empty cluster from a random data point. + newCentroids[c] = Arrays.copyOf(data[rng.nextInt(n)], d); + } + } + centroids = newCentroids; + } + return centroids; + } + + /** K-Means++ initialization: pick k initial centroids with probability proportional to D^2. */ + private static float[][] kMeansPPInit(float[][] data, int k, Random rng) { + int n = data.length; + int d = data[0].length; + float[][] centroids = new float[k][d]; + + // Pick first centroid uniformly at random. + centroids[0] = Arrays.copyOf(data[rng.nextInt(n)], d); + + float[] minDist = new float[n]; + Arrays.fill(minDist, Float.MAX_VALUE); + + for (int c = 1; c < k; c++) { + // Update minimum distances to chosen centroids. + float totalDist = 0f; + for (int i = 0; i < n; i++) { + float dist = l2Squared(data[i], 0, centroids[c - 1], 0, d); + if (dist < minDist[i]) { + minDist[i] = dist; + } + totalDist += minDist[i]; + } + + // Sample next centroid proportional to D^2. + float threshold = rng.nextFloat() * totalDist; + float cumulative = 0f; + int chosen = n - 1; + for (int i = 0; i < n; i++) { + cumulative += minDist[i]; + if (cumulative >= threshold) { + chosen = i; + break; + } + } + centroids[c] = Arrays.copyOf(data[chosen], d); + } + return centroids; + } + + // ------------------------------------------------------------------ + // Helpers + // ------------------------------------------------------------------ + + /** Compute L2 squared distance between two sub-arrays. */ + private static float l2Squared(float[] a, int offsetA, float[] b, int offsetB, int length) { + float sum = 0f; + for (int i = 0; i < length; i++) { + float diff = a[offsetA + i] - b[offsetB + i]; + sum += diff * diff; + } + return sum; + } + + /** Random sampling without replacement (Fisher-Yates on indices). */ + private static float[][] sample(float[][] data, int sampleSize, Random rng) { + int n = data.length; + int[] indices = new int[n]; + for (int i = 0; i < n; i++) { + indices[i] = i; + } + for (int i = 0; i < sampleSize; i++) { + int j = i + rng.nextInt(n - i); + int tmp = indices[i]; + indices[i] = indices[j]; + indices[j] = tmp; + } + float[][] result = new float[sampleSize][]; + for (int i = 0; i < sampleSize; i++) { + result[i] = data[indices[i]]; + } + return result; + } + + /** Extract the m-th subspace from a set of vectors. */ + private static float[][] extractSubspace(float[][] vectors, int m, int subDim) { + int offset = m * subDim; + float[][] sub = new float[vectors.length][subDim]; + for (int i = 0; i < vectors.length; i++) { + System.arraycopy(vectors[i], offset, sub[i], 0, subDim); + } + return sub; + } +} diff --git a/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexTest.java b/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexTest.java index 43a7066a64ff..831877101db6 100644 --- a/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexTest.java +++ b/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexTest.java @@ -107,8 +107,8 @@ public PositionOutputStream newOutputStream(String fileName) throws IOException }; } - private GlobalIndexFileReader createFileReader(Path path) { - return meta -> fileIO.newInputStream(new Path(path, meta.filePath())); + private GlobalIndexFileReader createFileReader(Path basePath) { + return meta -> fileIO.newInputStream(new Path(basePath, meta.filePath())); } @Test diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java index f4bdef48b125..296f288343a8 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java @@ -44,9 +44,6 @@ static native long indexCreate( /** Destroy an index and free its resources. */ static native void indexDestroy(long handle); - /** Get the dimension of an index. */ - static native int indexGetDimension(long handle); - /** Get the number of vectors in an index. */ static native long indexGetCount(long handle); @@ -80,12 +77,71 @@ static native void indexSearch( float[] distances, long[] labels); - /** Serialize an index to a direct ByteBuffer (zero-copy). */ + /** + * Serialize an index with its graph adjacency lists to a direct ByteBuffer. + * + *

The format stores the Vamana graph structure alongside vector data, so the graph can be + * loaded for search without re-building from scratch. + * + * @param handle the native handle of the in-memory index. + * @param buffer a direct ByteBuffer at least {@link #indexSerializeSize} bytes. + * @return the number of bytes written. + */ static native long indexSerialize(long handle, ByteBuffer buffer); - /** Get the size in bytes needed to serialize an index. */ + /** Return the number of bytes needed for serialization. */ static native long indexSerializeSize(long handle); - /** Deserialize an index from a byte array. */ - static native long indexDeserialize(byte[] data, long length); + /** + * Create a search-only index from serialized data and a Java callback for on-demand + * vector reads. + * + *

The graph adjacency lists are loaded into memory from {@code data}. During search, the + * Rust code invokes {@code vectorReader.readVector(long)} via JNI whenever a full vector is + * needed - enabling the classic DiskANN architecture where vectors live on remote object + * storage (e.g., via Jindo SDK). + * + * @param data byte[] containing serialized index data. + * @param vectorReader a Java object with a {@code float[] readVector(long)} method. + * @return a searcher handle (>= 100 000) for use with {@link #indexSearchWithReader}. + */ + static native long indexCreateSearcher(byte[] data, Object vectorReader); + + /** + * Create a search-only index from two on-demand readers: one for the graph structure + * and one for vectors. + * + *

Neither the graph data nor the vector data is loaded into Java memory upfront. Instead: + * + *

    + *
  • Graph: the Rust side calls {@code graphReader.readNeighbors(int)} via JNI to + * fetch neighbor lists on demand during beam search. It also calls getter methods ({@code + * getDimension()}, {@code getCount()}, {@code getStartId()}, {@code getAllInternalIds()}, + * {@code getAllExternalIds()}) during initialization. + *
  • Vectors: the Rust side calls {@code vectorReader.readVector(long)} via JNI (same + * as {@link #indexCreateSearcher}). + *
+ * + * @param graphReader a Java object providing graph structure on demand. + * @param vectorReader a Java object with a {@code float[] readVector(long)} method. + * @return a searcher handle (>= 100 000) for use with {@link #indexSearchWithReader}. + */ + static native long indexCreateSearcherFromReaders(Object graphReader, Object vectorReader); + + /** + * Search on a searcher handle created by {@link #indexCreateSearcher}. + * + * @see #indexSearch for parameter descriptions — semantics are identical. + */ + static native void indexSearchWithReader( + long handle, + long n, + float[] queryVectors, + int k, + int searchListSize, + float[] distances, + long[] labels); + + /** Destroy a searcher handle and free its resources. */ + static native void indexDestroySearcher(long handle); } diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java index 1a9195856b93..867abfbd9881 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java @@ -96,11 +96,21 @@ public void search( nativeHandle, n, queryVectors, k, searchListSize, distances, labels); } + /** Return the number of bytes needed for serialization. */ public long serializeSize() { checkNotClosed(); return DiskAnnNative.indexSerializeSize(nativeHandle); } + /** + * Serialize this index with its Vamana graph adjacency lists into the given direct ByteBuffer. + * + *

The serialized data can later be loaded by {@link IndexSearcher#create(byte[], + * java.io.Closeable)} for search-only use where vectors are read on demand from an object + * store. + * + * @return the number of bytes written + */ public long serialize(ByteBuffer buffer) { checkNotClosed(); if (!buffer.isDirect()) { @@ -109,12 +119,6 @@ public long serialize(ByteBuffer buffer) { return DiskAnnNative.indexSerialize(nativeHandle, buffer); } - public static Index deserialize(byte[] data) { - long handle = DiskAnnNative.indexDeserialize(data, data.length); - int dimension = DiskAnnNative.indexGetDimension(handle); - return new Index(handle, dimension); - } - public static Index create( int dimension, MetricType metricType, int indexType, int maxDegree, int buildListSize) { long handle = diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java new file mode 100644 index 000000000000..f818489c8fd1 --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java @@ -0,0 +1,168 @@ +/* + * 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.paimon.diskann; + +import java.io.Closeable; + +/** + * A search-only DiskANN index whose graph lives in memory and whose vectors are fetched + * lazily from a vector reader via JNI callbacks. + * + *

This implements the core DiskANN architecture: the Vamana graph is loaded from serialized data + * at construction time, while full-precision vectors are read on-demand during beam search. The + * Rust JNI code invokes the reader's {@code readVector(long)} method for each candidate node whose + * vector is not yet cached. + * + *

The vector reader must be any Java object with a {@code float[] readVector(long)} method. The + * Rust JNI layer calls this method via reflection — no specific interface is required. + * + *

Thread Safety: instances are not thread-safe. + */ +public class IndexSearcher implements AutoCloseable { + + /** Native searcher handle (≥100 000, distinct from Index handles). */ + private long nativeHandle; + + /** Resources to close when this searcher is closed. */ + private final Closeable[] closeables; + + /** Vector dimension. */ + private final int dimension; + + private volatile boolean closed = false; + + private IndexSearcher(long nativeHandle, int dimension, Closeable... closeables) { + this.nativeHandle = nativeHandle; + this.dimension = dimension; + this.closeables = closeables; + } + + /** + * Create a search-only index from serialized data and a vector reader callback. + * + *

The {@code reader} must expose a {@code float[] readVector(long)} method that the Rust JNI + * layer invokes via reflection during beam search. It must also implement {@link Closeable} so + * its resources are released when this searcher is closed. + * + * @param data the serialized byte array (header + graph + vector data). + * @param reader a Java object with a {@code readVector(long)} method, also {@link Closeable}. + * @return a new IndexSearcher + * @throws DiskAnnException if deserialization fails + */ + public static IndexSearcher create(byte[] data, Closeable reader) { + long handle = DiskAnnNative.indexCreateSearcher(data, reader); + // Parse dimension from the header (bytes 8..12, little-endian). + int dim = 0; + if (data.length >= 12) { + dim = + (data[8] & 0xFF) + | ((data[9] & 0xFF) << 8) + | ((data[10] & 0xFF) << 16) + | ((data[11] & 0xFF) << 24); + } + return new IndexSearcher(handle, dim, reader); + } + + /** + * Create a search-only index from two on-demand readers. + * + *

Neither the graph nor the vector data is loaded into Java memory upfront. The Rust JNI + * code invokes: + * + *

    + *
  • {@code graphReader.readNeighbors(int)} for neighbor lists during beam search + *
  • {@code vectorReader.readVector(long)} for full-precision vectors during beam search + *
+ * + *

Initialization reads header info and ID mappings from the graph reader (via getter + * methods). Both readers must implement {@link Closeable}. + * + * @param graphReader a graph reader object (e.g. {@code FileIOGraphReader}). + * @param vectorReader a vector reader object (e.g. {@code FileIOVectorReader}). + * @param dimension the vector dimension (obtained from graph reader header). + * @return a new IndexSearcher + */ + public static IndexSearcher createFromReaders( + Closeable graphReader, Closeable vectorReader, int dimension) { + long handle = DiskAnnNative.indexCreateSearcherFromReaders(graphReader, vectorReader); + return new IndexSearcher(handle, dimension, graphReader, vectorReader); + } + + /** Return the vector dimension of this index. */ + public int getDimension() { + return dimension; + } + + /** + * Search for the k nearest neighbors. + * + *

~626?o|Mg6}%FnFtORUNms9){BEOPK!6ks)hMJ#QcBpXu7Kz98X}LpLjsIWUgyW zYD*>W1pWeV$q7`zJbY7Jm3-hT`KvnD^}wl)&6%g4I$-%BIve!blgt)J>=5-@*-88i z858;R$YaVL#GF1#n+?o?FP@xYKDYd5#}*57_K@D!o^vxtK>xOzv<-3OFRwFcj{eX+ z6pqeu}DTrC*_Y;n$>lrL>s`-KzpFRfVJ4L}!*s9pXC_ zh%HOj+5l$@xGTKZCTkx@=uY(ETIk*>&>!vTzNe_-CI!^g9 zkABcYk!e(@hL7{umokDUI5Jdd_4|9rmdR_ zVz-7+<|F39tmB2|&cJq2+N~{^d8Kc8G0Hg+{1p0E%bp3P;cA5)*|SCRb)%$Jz@IF>=T($HNGhd)>c{gXJ7q8C|ONNzwc{X2;-*sJK8 zPa~f?$yyW|D0TG(ej}(asBl@Zh+VcS}0?^1|1aqvS<;F7K=d# z%Vpg`2RRG-4 zYr;q81EKRzP(B#kY=Q12vj3d~wzY2=v{7;T%B9^ zu#P{y`2JVW%eVOMWpJf3_;yF7vQ$4(cr?WydN~9;iacoLv*7J+p63-_pR%jqhLp8R ztB#ZU1N|B^8m4cZ*sMFUI(oF!PiRVuh-SNt(U>oUwP?du9r^A`l_V;N@*Wl zJpz7QrG;Vc8NmNqz9K#tW`Qq{$0}E$6`lbaxQF?+42K3XzkAt-h3=&fQF>0(9qTTy zDx?kZOD)ssUpTS_i7yrA-hPAFz)BAG{_Zd@E&L&6|0nT;_b&e@Pxw!s@GAYgm8CtSdscIOEfFrS@y!v9HhFakXK=h^t5 zXJa29cy7Nz2Y*+4l@8W(-4h+$bp8K*I{3)4|2OI2I_BZO^K6{ToOY*!sm$lMp@Y$! zVWuM^T*n#NCnL;6MmUgj&lBBbgr&#`3%^E2c=#L6%B$eFMNamuWrXoxl@Uf_+uhC? zCWkt!9*kVA%`oJIQQ%Tfv=H0eYsv|W{M*J%?mh~Q^n6Lg04qd4nILBX;PC-``!~rA@B9Y2VNd;!^U4jguE-5fBJ#xPONC}eb&>6X zbKT^LQMx?w9nQy-kPY6dMPHF8x(XA}bwpLS;Nxw`6UXVY#Kgn_t3|FDgD;1~ssGc9k!7?Sse{kNadi*N`uY@7|X!c2_fNB|(dBeBxgW`QmEii%;?H z>X|yiJ%#!C+iB?4x*j~)u~~4U*<7|if2d`(oUaSKGR9ou0(O=$9{c6xRU$|1EMq(t z2Oa&3A!EFUcaj(5v%AgM#Sjm|tm$!sC6=?tteju`xMS5}{$*Z7kB6T)xgCv|5m)k1 zRA|_}k(&>lb)7~YkJv#@4>Z9pIfkB7^>$0c_$G73IBe zl%|TE;|^8q8YjL2o}tW5405N69Xx9orzur0c>;a80hzBHjqiUHqgKYKm=8UyPi%Br zcZfV49K$!IU8O7ML#2ZEIbanXSTHL@CKYz2^&b)+cZNHTm7VDUUjI!H+ zgT(H@j>X&FJl5k_WN8*?!lU$EavI3IivPe5xVK;K*erP(d^rgvhtmD@>9d2x`9{Wl z$YFOK$rJcWJ`nm#o89xe(06^TGB%kD#hMwl!5H6p=HMmzVrkgkmMOGB^nAOn+DtO% zP8aLYGn5<_z{SHpf`0i-Uc+eTOZ->n-r_tw!N{*0KUmtnYCqUL|A53Q&+6z{Ma~cK zx>F7qb5|Vj^Q^DWGCnC3c+ZBjmP!9JK+=E&!Fk>ic}$Xh^tM&5Scymrq) zBD{t#FF`)Mh6Nm%LA~JT=3HdEqFa!>F&Z$Jx%iOhX0DAj7U6FH#pP9E_akt|_Mx@T zR92I}pB$^`iqZe$6V^RvN!RuJd;F4!X})0*d%OqSkhxIcMFF%siTvx6iEUD<1-29t z%WREbXsd&977+&^7@S^%zrKUK?C6PGrV;0?Q2Ak(Nc?QGxe>iz_2h*gvJVBdq@&}{ zSQx(A@I|#WN{sG0p1s3ZME{n!&~No|bV<$Njrb!q15bgY;LjoMe*-K%#Cn%p1P!vM z!As1G6WDg8Ej!6~l3^zXg$-X=#^Adz;=buQyUTa~DEC6gd^Bti-&xk84r_j|p>oq0m8+5dEU0j<%XhfVlVX-ccVc}2%6?3VVw zaQXb6?IwIcfu;0mDbM9!qYlwEJwhGlz$c@>S@dZVb`tnZ?CM0DC_4~1#z0f0pPj$I zhwlYH{aC}kceipjm$)In<{RUg=A5gy8}9ysb?)N%#kp@_hoNDQVWK@V?MlocW3PyW zKexf3#}PO00&7ObXMsPDV$I+L=Um)LmnFPNl9xX(V2qRCzr=n)*WshPoeAz)8K+Ng z{{S-nMEdpwIKSpA`m2a*>aW(IzuJinQmrnNmOKdI?tzSXo*#M-;QXJ|`CrlpAGxXz zj%ExG9((e@J>X)gIpT`$W1cy}u?F3TD@x|l?4I`8ed=r>_tiK3`fS#cch9Iyjy>E!-;p#-uV2IU|U=Z$`h+DtaT;w|Z0Ih!la_uT6a&XLKD@c+1V`pbq2T zd0N*&g}OHpW6@?Zbnu&^`!tGgEk3)AwbA(CsUK`zyExqykFWYf6^t$}Xs^Gz(6~S_tD%(av#loB=>{4AH@A& z?gy!M!JVh!D+Qkfj)GIKf>VM|r@$xrQ*Y{>UY`-H)zhE)e9Goi?pnIY`TChE^2(X2 zd(LRl=U<$=TaVKrInHi=(V$gAJ8$5+p|9F}FEl^1ZgSz~V2*m2fa5%&((*ksn&2G-ax*4ReY)JE3WFxFT-Yiud^!&ze| zS!4OYLEjjk^^4Y=JVezf~)Ih!5ori)C3=jyJDj6)Z>i+V&CdGsgg zu4h?`uc(l{m8{PvSeq}iM%RP)pWtu&t9Ret`!M_d+x#jTcI|zs?5!6*d2_e^>{%7D zw_@Y2y+_O5exZ`Jxf}d{>)p4g$Lx5;TmJWDyD9&cUjA3|%v)Z{Nxgq-IsIE{|IN2v zkUH|pcJIwoy{KbAb(vRp>JLoub>7m8xvr$|mEGR$W-Z>%T70>SZgoKQGw4fuTNVy7FKR<8j*qjIrPrOcZCVukdiX^iY z8#e0|J?C?)FRz-a*JXxJxSjovSabJIU_Z=YUo`OcFTcKw-<9NU?RlT{z&8;O$km@U zN?ythi;%_a!RBKCYZ_acE_O^E>|f2q&$ana(rw+6*kiDB-Lnl(Qc8t`jqz-ef}Cf4|J*7(h=@iVOPR=$ma4%G6j z7CI0E9hkv4<u?aiY@ZsLg zi#~ecmhFGq`x$W%o~6ulufDx^bj6=vSUFhMX@ujWcR$kKZ6#*Xb35POyPI_yzWt-U zpR#U#V(R7Cz47h60Tq9GVe|VR?tNPYIDSUmpEm5?`wI1jZ#SO(M4oNj-KCuVjB>j- z?tWqSn;%l|NAx#>I;_|@JgV=D5sdddHV&JyA6wb4;p1b$(;UZmKd+=xo8`Rwe({-EVuk;{B_t@`8-V(%b0 zgWX!=<3q4_xLTeJ;d*kjvbgC+6@1wZ^A;}Myc=p--P+lxL1z~7UD3YmwD zD|{NUH+Ly#{^OiSo=fN4qUxIlaMtF`Lmf_8!^3E&vmA`NuE-EJ!*l8OHt^x%J2(m$ z$~T9vGtPTmMq*y)s5ZqSLIa`x}KMPG612oxAARfdDnwIeQ)k6L#Pv(*N#+O z<`v_WdBq`5Qd}SAe0RwF%(x)>t+2J1{X~;7bdz~CRs4RO3N0J&8G*bYL1bSI$fb55 zj}qBeS9yajN20Dx1C6tqelCp1X9$}bfz4`Qlf{`<zYK zmwM;HcfXkXj$Pts8v#GtAHH?~{B0!rKot8xG_16Lfb|>VyF2IOsmP?N?a%Ukv{pz*pNo%N8u7*wu~Tn0S;pG%&9@>eO43SSH_C3fQg#P^@x7@B zyVSnKu`4!-Oo{pvV)sNJOf!FEK`yyN_b(XW7F)>mKN5K#Hi`QFt=j7rWu!{HN|9ft z69X+}*Y>tZWONpM96e^s7)Qtf})sip4Oot_H6F!nFIwNwvh@Jx6# z5AuQj>8kw#=bP?*LZ&cH; zzv~8*#6HEB!TY?bgSgP#JGu7&AB)g+;S*jp_P0{~x#$)&=M1jLfEy)@SLfk?bG#Ya z0Gx8qb!_>Z?=SG}fOF)eUvF~GslU8xfVaL#Z+%6b>br~Ui|Bl$tn!~wpWDk^uly(S z@5g^S{~G@p{G0f<*Gt{-SoLLk`vcr=#{V|gmliz08n~aea35>pUe?C|oxG4qBI0}=(heNp0cdIz}o*~Mu47Q z9G%vR4|i^Fo5uR8iZ*b+?uI=z<>0HtG0vb4EAw6rzE6#^IDY(eGwl&e8=3GPSp!;l zOD)&=ruko4Uq1MZW8MTNpE7sCPyHMBzIQ#fS%IJDM0}9tEGoG9OK?u&;``cL%ll@Y zk6vnT`!Je*-zNP~EjyqyLJO)h#OF%27qHgS=wBJ@r<%2wIo>E2!+ln|y}i1lV~fx! z4}Dw7H}fLZ=Ec;-_>4IJn#tL{t{u<<@oDjY8vQ@AGyED`$|D(n9KO!@IA(Iak2;&t zg`|;xQM2(*@Qk_3D0q4)b+0XD3UX5Li zz*=BGFeI}rP3zyHu_t_qTs5D0lRn#-&oy_5ZVS5w)=tY<{AGv>lE?Gr^%*Yl`<3-L zpZV~WyQfpR!|X${cX=px9Q+YmnKX27itocq65BF0=pQnYOmIc-6ZQw<11mYcZ#Sux zzBbMS8{oSl-<014#~bj~_TY1*pS|MC#>?4jtx2nIo_dE%bR7}pD{DLzRW%-BGE^;+ zSl~gf^N%9KWggeuV2tC!1bmC0wYQbBU*tlo1L(iNIFWxk(;5p*#P|C=`PHGtIiG_w z7xGm5x&Oo7`+!GP-3kBa+__2aBoi{oB$A|@7#Ot`TITJ|L1$Y`<;<*-V?cjZ?gF26>L`ZM%q^B|7qdF z(#~V}?z?Db^PBe{7Jt>gpPWGlUkc5+5!$l^8uSHd(GAeh>!G3HdmYX<-c{5m?Og2J z5{L3k6I`3jS=%eEMb^C1MOLb{Xw77`Z*9lUrE6E$f6IA)U}wPx-#VOFTDkPNuKQNr zuEg;M_)>O1=TNYdubQa#Mc1>Nv7N6kIq}h-ao{RSDpE8vvbhR3KfHS*u zx@Jdw-s5ch+cD-8eS7J@dXcv%^D}bd%uy;_#&?OodoI3J%H&Yy5M^Wz64(Qq!HY

h*Z?uzy!bxYZ&DI4Wo+am3}0UrK=cB0QE z-%K^{@Kbt(mIm1q$YS9&Ca;3V7AI@$V~wSsr!qwMi-X^|S>v&+^(CzNIM&_+zey#o zBu?Tm8n7J_tT|cy!#j50{8sy=5f~YG36iHqWqtu zEsK3bUgEzGvhInLO(MU~9q_L{XdHd}d*Zv6_aU!|9dPUYZ?ruFzba?_Rw6@|Ua=$d z;fZ0n5MQ`07bYS%iqAS)??ccS_UKi!DZdt8lc;^aI>CFn!|V3@8`~bGz96}ix;gI^ z`R-T2kPf^(L+uM?Lr;(=CB8;vB9RM!v$81A{PoWT>d@hT_;cpz826U^;>*>6jTyNR>yG4-~cV_d|i?uhu*;{w4?pM$f?6UG_p7kr!s zF4#5-epk_+^yOQuc`nZ{bZ=cL-(_#CZxb0|GqIQF@}EyCBit(Sn~eXzA|uQI&tqkT zU-5fD@RIRdNJhAmJrI=<{#MKP=a&(hoTD;=tUvVRLoz}cc=ZCdgQ$#9#+omR$OwrW zM|r)-;!$2-37khCM&t+h-bcv~-N-H?KfHb4Xr1NY_n%8M$I-WQX=V-Ue?FSo!Wg48 z^G?dhd`DwK-dQmRyet7Pe_#J_hMl|PL;S9bIXPc>v3<%qo$uZ6kmKatZ&{yi_CYKi z`)WG03438QuEgF0t2`HtD-j(3oBA``9v&yUvCuUS`wBXCAGxR4OR^WN@;OEMl9&`}r+{|W(vFNR7dd8N3wdh!zn}KxZwl)w zJn>ET|Fg_@n^ISnd=T23TU}_|gBMhXw(aM6Iebvva7ze|hI66*PY z@1>qD{>uFQ6@M@0M@yX~bTd`X7oMIAo;byS#(tWO44lt7b(5TSW#pKLPCQb;+5M}K zyDa#il;1-6w02)$i0@72F9V(;`oGxSLq^(()#S<%-;%_k*!4>;dDiS%rbI=zIZiL`gAL6v9?C&Z@9gcz-8R${x;u8KZXCu9EvO_eXAtq z%T&PrX3*g$Ua(_zPP)i09Thnl#72_; z3B37d%86ZC=JoYdzH!K&Cg$}y@)gqOV74j>O3YZU>F>=Z&Nq+vL*m8)_z#c&LW|w8 zV@_SBUzHHQHW!}~F*$Ot$oc#+1NcW|kC%d97Pbb7wKxEt`H)!*VytA&4Cc&Yp2Zey zM%RRQ?~x0{;JE^}_%HE)mOeNBFD9OGi`d>g(8N?|V;a0W9U9_=)?j~|8*kH^EZS{> zcPlk&e-oTU|5-Z3IIraYSAehPKj#?w-{q`x7P8Dk4&SeW+@H19*4463l%dlPgYR<2 z{m~Te1puG2r-W^1*(qxM&m@;f-qEvUkSiEV<(xTTTEP1dGTb|i|J8`?X^rFq!`AcA zyWe}Wj6G(-_nx1xhT6^~1S%7JkB~QlGw)dqYtD#HpLIfVSaj2_*oPe8>iO(Q{{hXG z=TSRSh~F~Sjp&{tJLM0Bngplc;CW7-_IL2Nl`*|4Hn2!v4j`w!#!F73_-l=U@{GZpC)gy}4L-KTw>5%}8}`0j?!G=58EvCyNO#~v&W(9aL<`!aZN6Zo(Ky!aCM5#cM? zW6$*;iYz@FJK=N04oUv|O4GV35u2&lZW1@jyj!;3EBKl?gw02uM{GJdLhp(k|0Lgu zJ!l5@i?{G|OP-kP=5ZFJL=D}|v*YNvKPC=i#x>ZoK1jIg_Bq@~M%)H+@>%tjv9?l| z^hfgT_1a?@#rMg4U&>lq?lpr#Cm&$#qI&5n;4mKzm*1R@jNqG7R4I2=z>5_3{MtUy z=frf=v@k z68!3ex7vuMJ`EkQ`6hPtCJVgD{g3aC@He4t2gwCv?-6LW@UMTge~-q<+Vg+5Jk?>X zns8BhDiz!0A^y)A3*C!Ih?s)-kEH;cISJ&k9!d8OUw#Ai(0YQ&#v%W$;iyjJ-q^LO$GK@0od`pxiQd(L*5 ze&>N_2E`t__5t-Tzut1`;3n?AR>%pmH^zN%yp7yv_$?9}y0KZ<^f_mYKe$@?%V~EF z^roI~*2s9~s-iBQ?^>-6y*^j@pXGgUuCGY^7Ua&#L`PaL<1R>jC{y|nbUc)~#?Tq| zosIBhaQ`$mkL6w1fi$uA`MUfOC-_-#Exr=;)LzF!JC4J%oWs&5eWIH_`6#bLDYHuA zy|cJ;=;3qUi_K>&bx7M{tIVPORDO4Hjwkq-uc!^5LKt_Pep^Q#^1ZnV`jShz?BQK) zvxzg`MH{l0*Ic4A*34l%=ZvF^aYWA-M19Nly6uXtn~vz9j)3Igv3Y*i#F2SA#+*sN z@JG~4Mi!U5qYL>f{+upo?Jm|e2n|PW9V&f4;VOI25J&mXyqEQrIe#m1cHs}m4Br+U z+A^9?{6D!9$Ed8qbNg&VZyxf+r@>L=iy-z$ zTfXoW_*F|-zWA$Oo-JQQ?{=yaK3?YVf6B(^i|0s-o`0P9>HqrqM}yeki^YveY?ho2 z`=sX|SLdno=1BVz^|hsuBW-i|?Bi;ME-Epi=jKSe$l1peoPC^fp0kgSA%{GlmUys$ zx`sYC4b7tWDZ${8d%`?!exjxA}Zl(T1(llhH}JwG{o7P1|@`lz#z?Z^_J zIENZ{y8TyjsI{eK9L!`cB!}8#^f5p00y)&s?{ebQo{!3*cJ3KSWb^;lc)<+f1=H9+ zvtx3oT^W-@P2vUDJ{LYGF7d!q{lb&NdDJEXEBpLo264ZmdDJdrf2T!a1<%c+mMP~S zPbqZ2%m(alFaHgAK8&xkJRuw}C}$hJk+Y9M&N;ltnhzoab)9qekuvt#M=Nsn(axck z9?qfmcu4etv-6o9M=n`-fgEa1`dX;HBWE5zltT?#Uyz!4Feh%)p87QFpo*Klr-*O9 zjQ>FBww#stujEjBe_b?(8s`!tIn?BgqvTKK^q zOL$#4huUaeH01tdY@2UBr~LOtz8|!0 zVCO#v74CQ8MbAN9BmGCN6#K{wf$>Y(ZJcus>SV?F&p}=H{j<+O`R5;!_L`W`Ceg zuLUzm<=%Zs5n0i`zpN`Myl*7mxRWz5 zvRBql^cCF>Ej#Y8ZMAZqu?X9uZ3DHiDVFLrgCdKsVje|j?s6#q^v*Ne#Gm`TR#zT? zUgW9*TUU$U8r9Xpamb9>*4q{PK+~@)B6G<+_p5t!c&Y5i&m7CRMe}R;V!HFPqVXexF>UMiCrrYH* zuGnvFeA-+XLw{}k@Pc*NG`2--&&ofBx}tjpS-ru+c4y@hv!<-ApXJPIXs$XW{=+u$ zg^|lL=|K35>Dv2`TPxAM%2?ZjyjzF9zla>|E757hrYtdK@?Er?=uZ#*VDrlb|MTZ> z_Wn!zn-9J4?wewByzYMI&PC*ub)5*%=qx>TIjn z9^gHC3^0G?w+%nP^OyWS3H%bzUIczorvUR;e%tW#J3IJI&dV42d3Fkz`!<1{JA37~ zL;vb>RsQB8ozk$5|B2j*Ucmc@fgk68JO2yp=Q`lJ%m^%GZk^hZ!TQ+yMD+Ejt`0A> zb*|^-&h8SqOPf2nnQOU|JB%w=`{nx)-U~j9PG;kUDzf+-$MV^qB}{vk1c1(SQ($-`|GRaJhrES zxfwZ+-34tIJL&n(V~@n(Qg@8=QA7IO|7+(l`j@WWolm6T;r@dG6 zXkYey>|6=mYe8mG<)0ggHC4=6wna{4+EhMu$mRnoe{hXs@HfKaOR!7W-*jCy@{P= zq4sO&uJDC5)K|t@hi#N+pL1XDQZ;WUsTwI4Bo71Qu+M*7mi+PO<27W_i=DR)IT8-Z z{MhsP*ci;jDS=?f9Vl%Er|@ems+$vNcx-y$XTP{2@XVG=221CwCee2!PU8$vJ_?b<$R7;=vPltDz*rI=Gm1 z{lV##g|j~h-wE;z{F^(hmOnwx`xkSc-|^2nz2Oa2IID_s1rxZ~jlY|Or(?tWDSqUj zBk5oxbGQvTHQ-n}7$m2T*afn|kz28M6~wVmxL-kCmblZZ568f61~2QkEFH9{tDbqc zI(+wL?<205+{F!n(6(Lp>?JS9``O4S3y2rFO6^Nye4!O;-_?B^4;MjWBHtg9?+f^y zpG^MctH@Kh3(XW+^EmMho?+Av%0kx6T)cicD_yJl^oY6KQ%(M^E~Dj|dg@cD<=32B|6>m0(0};YCyr%<%-iXl%Vj>7Z^-_l zUJvbG&Yj=up>@l%=dq^5G1O6h7j>EQ<(r!ZE%bZ2Uwkuh504tDC*~H)_csmpex0>~ z9|gb3{$TwaJP)oDpI+kD>$v}76ZY3ecsuLAK8HBFP4IgCIsCxjLjgGrEbY7ZKISdj zUT|Kxy-JJrCSXgKzR~W1!En1;hQRDEa~C-CFnbkk4&Yl^jeLwxZ8hH<|Ma4Ob${Yc?h!bVC%F4vuOsj^^=~xV!qfX@ybz zduti^ck_n8Q;shWVr#eKqhI9?7TK32m#O97artJwpY0o(&$#4oZ~=F{yq++NdiGUv z&(u44a{}4Kz)pOf7`83pdxWBN)zd-HAx1RUvu z&p5^nX4C#IspBz6cLWii_z&M|CoPplNx&Z4Ee)2s~}^13RrTx5L>@So3U~bn^aTF11kr8 z{B1+-s24q1>Ud+L9iMrWI@-|jOOZ!fkRNsLefIO0dH#mr>EDFgwvxl`ii|6^q8E5C zHm3T2xa)9f_J&^*t2jATbFVsWlqPYX8ts&EE;n^dXq)VPk$*)d@>Hq8Z+}$>Hj;M& zS;c?8*pXJA$Eau-bt)OwEnv(|6BI#(`QN#KY3=!;q%S0^>)InHOy~4wBgB>GXwh_se{6c zhf~=9;`fZm*7cvBVawLfgk}cn9iBmZ4o{l&&oHK7hksudous~klp3nSk5WAA2q3AEArA0H1+9Zo6eMDL#=cCxLD$M zla*_0o#yxE&Bae3%RP~*h8BMiYTC2;tAkam)FJui7_`YnIez;kmeyYP^60v!H9UiS zFS^FhkUhGQkv>O0{jX3?Y-}Qv%bc%E3T+d9DfCjF#jg9?50BdZ4!Od%p~hec zFY$~$XWv~W&&0NMA=>adb$Pf;OK4YVq^=)Q7dlUo=w?FKN)w@B=y2H)oh{af>EoSd zzGu77#wTR9b9FUr*I`|)nel&AjJNc=0KF?8n@a&}F+G<%_>h;ke+oG{PpyB5F+I$F zxIsU(pu6S;RSCwI|%{la(Ydqii!A6@?Pcd(TWL6^klm%;BO>X?eI zdO5i?{MuKPMmz8Gzw&AB+#t`*TncO?e6Z;8gCx9YZ>!;bdIltaa!Hd(agh124YAJ`!Aig|=h+HUtjd z^UPg`JFa2Kv~RXZy{Y%AmJ($mn8Ytv`n@e~a~7LyjJKUdH*JC|$mT=dtuO z6?%Hl*U+2UPv4zr1U7zEay(u<#`_`0S%7YYZ0|qrnAXrc#R&W~7d-;sQ6jp8$UmpH#MTy<(#9}&31gWUU$Qrqk3H%@#h8Tbp_~k=%Kb=Cu=70MnUFPeV@FX zd7=lEFAe=70o#FvpUO#2Le}Up?q7Y?6&N^`9Egrl&dNCW|GS$*+uqqyQuNv)WK8&H z_cCw@Ta4gP@bH4bI}3!jrkz-e94Wjt1KHy@=tphwX@xyUd7))=Ojj@_;Tx%pF?KH2^G$3U z7P7F=IguH}MiSe`*C=}wzWFMAvj>?{#yNYrIz5qo{ab2i+yA1^A}<90IUz9cC(EW6 z?>*)W48N8bkUjOyunlTNZV0me7Hxz!IRof*VLPeNdCGpC{zA7plq&E~1SbE5|4@2p z+xKZB%+q8&!(;c+uXm77WvxZ#6knG3rbo+VG{{+R`F)`^yj_kLjIJz z&BHX9H67ouf<7Fe4-)ez<9-@BEtu!FK#mTn>Cz5w|Z%UcgD#Q_v9z^ zn1=2LT8m5Ylgb!F;5L0(+V@*mpzo4vi%zAP{!`dLO5n%uF2JAg=8uZU3w=1{gij;} z-VY@OPCu9&czx#>_`{Ei$q7Al5?cKB)t3ZL!c*S4dR*WnJpANiJWt~9pFDw+$Sfz1 z5!22Zy!BRE;3VsI@)_P=&fn+wJB7cGc>}#K0Y7N?pFRDf;(;mS11FAg@ApByuWv>| zk(~WMWvG3BT98nbG1v5mLK6Za7mLsL_%9rOAKw(SmqH0zbeu;*p=@GOtfIaJR#9lF z*&IyKL%rkl5IDZq+U4~JId@sf9IL;2){fJ?UK6`MV?ojEO)4OD_EKjKbxIxCo5n#K zspkv)r~c4&{GDRn+c&OcsQoV0gbnuPw^B-meh9C9gMIPKRA1p6w0DyBe&kRy`xh)I zI<4L79zeGVWvKNoY=t4@lBwj9V1KSJ=ibv0c0V6yx8?a}>^EXV(R1VcFCj-@*DTCl zs88o^Q_CB%wQU8b9s;K-KPWrA_=>@&r2RME=Pb6NJTI2Xy^`fG%QIQmZyKpvPsRC) zo{7k~(%zaf@EaM|J@Jn_khdFBka71Q<8uDj{~CI=*uPIx*Bf&e6eZWHnN!hGtDJw# z{K4;&$Df@2V9_7-w8u@(#GIr}WazDLysw7dm|I$Or=@0X2FF8}VM`@8HZ%cSYAS!R z?Wytn<@qkoj=zRY^`S@9@~z40&>dsY-|4-&!Vh{qa+3mW+XY;Q4`0 zJ-CNEj=#ZJo^`*|yZ^x|o>%dFkCC|bP9u3MW&ASlebn{vcU2Si|3i23&7Ndz z;7(uBYs~RfYzVZ!zD)WdHr`xY-&}e;Z$Z&Yu>nuwoG9*d56SUqfenW5$nmMCT#dQdhlk)qBQ*VL8 z+O@UM`QnZr{%*`pvF(SjVV03UA(X1t*CE>qer#sWWbAzlN{gzrS}Hi~ld=WWg^f5g zjrk-$O6UsW5%~L=8{tEg7eC1&{$H&WL&{-s^vFC{2y)FYM7jZ zeuI8c5Y}=0&qVwm>sfzj)ypT}QbSeuC+~dY6!D=1QiARI3#CPz4VZO2W^eVex60XD*`aZP?44>TJ1?&2 zcx0a@=7eoy(S0VrqkFBdC-h_k{d$dc%Ha&?Ys{S<$DY`%nxwDH!O{SGkNB4LqPxtd z50z`x@+pjWU2fZsZ$SH=&nuk8J%wZO`5bCI6MFJ9oFA9D9$-)Z34Z+|elnR$o-JL- z|0&ERxgq_hpi$G9%axJ3nWy#j%cH)e7k0>8UY}k#OLQ$_1UA5b(5Oe$K2nM z6y7_(rmY)%7Z?w;_uTR1-*EPL?i_di@f-*L?eVOzigwUO|ALzW$K%wEi?}y^Hu`(s z-D-IXdot%9)*romGP=o1#`unc`BZ;W%-n9GZ0TRKXKXxuVFxs( zVM;FR7MWX_ael6w=J5Pj)UazTjy_m7cvTf;XNG$aYS>xAxF&|PWoX2za!Df=_B z*WZ9fzDD2cz|E8Eqx&5i!G3>&{SLqJMEB9ppeYyHM{Qq9><5Xw!}I4arpN-d^l@&IUCY)pa1bDX%~8h&+nknPa{HK zLeTBW)GuqvU8DzvKe{4gOlFKHprt_v|7ReJa#!gIe1?}s%AdeDDl}W>#KoLQ`Ccji zE@h`Ec(C|St%G(QlzW_V0m{`=&c@NPVH|xUPYpRgs?HE~B6IiBzVzb*^qy0WaR>W= zPdmntYmt1B$P}lSswQMB(N&jL7-J8@Z~fiSrg{A<3pWjiX~HumG+~Ka{&TPIkw4@- zx1&S38{%}~tU0Fgug+3q$wTSS<^Q9cX()jYa!$r`+F>4iow1!}91h;~p)34}dsnl` zlP|HVCD;{ScV1*3+2NO)-P>|qJNuBS zUF1cb%6v{^K0Qt~^gL&9ovW}X-_QR;dPBBfH`i@>#rBJ(uXO~nKQIH_(>~P3ID*Le z(S0j=n8@)Ww+jx6K9I;929Xi+Ig8wjE`)8eDY)Xgz~ks*$61pTkH=x3LqEFQK1+uU zhI3ZvEDIwxCJ#Duew=fw!}i^zqSuA}R&TK{v$zl9eRMq)r=BciKHjCix8Ehk^{tY9 zZ@pi#?>Ozbk-E!s{Wa7bTt>aA#MV&%c-G?hFXjajFXOxsJS(_D4cYn`bw=k^ z>Taj*I}O)Xk%vRjE!G(y0PDv8jo7x{0q;Z~jeXw9yIUq~_;tZG?q&I#t%h`+MLZb3 zhe#bm>Gbo*5&POU>}%XrJ%{%PDzMu;8nMegMqgummru*P$0OhTa`c-fdHz)78}QF5 z`BaI|5ZmA$dB*Ni zK+{9dIWOwd>}E~HHu^JS8Z6p)=CN7CVWkg7WfGao*Ry@X8OJzTRzf$(@Ta~}& zR_%|@%|zDmpIKjtw-OpAdtP*Bp;Hd{P7-y={K(j!CH_O6ADfs(iZ7l==F*3ZBXb!$ zcQW69$NW7VnZK2e@pf*m`Ym4~cFH=Di_bvjN$hl!7~c`_rHMXo!|(B9a81@s)<*hE z-%od;x2*!tHiK_v;N3*Ver5~yIKHbt{2hE9=_eLKr=xf%-(Q6vdNtoiaqlh4i+>t@ zM(lUhb5x#x74H@6`FG@qOo1Pc*o!Xl-Fp)K_-@vs9{G9=_i(47^TzJs%jjS1Z`srD zu z2c9DI=D5&#*2DVUU5_lBu;G!1IcHL!6QMCq;>x{xGvAvymqs*p#Zt#H`u%+SB2yfkorBPTbq&f^+>LOnV7nm$Xpun#6I8;MUT>Vt{mTvx=0^U?!nwtbrS zLPK(*-yw@UZApxR_Pnmg#Xv?a7We1BcK?E{j& zTao&?S2FT0)6NP11m6n2%XdPHf0+;(`7Q1KB+{Sc{kNfi z8T29d-R~XlVt>woCe~2a5ot%>{}c7q$EVscS6$FN*-Kj~(E}d4xI0(jRrV+*any#hi#<{D0$4x!=T`VQ4`f2;y%!-x&rQ z?|rHMe36oy^{K|fk@m(zdHX}gzCft_F07L=vjnv=t(KG=b~p3 zh#lz83!g=}_bcQBlzk#?)D^~_cPOImqI&W+s8Fa{O?Ubk@5^=24Uy+7jr_rSbMUFk zIm#c7*ORlL^BLDXa?D&Ch~{VDyj=}3L#Mk%t`?fKYufqG^*tEAgKzLwp%Wr2qZ6|S zui^VJEj#~rVR>>IbWiyoV}5g>S8^UOdZ%1BW2-;I*|F13e}I^kz+<16Gd|T;_h@~+ zn>b&)nlnD=^-_;t)@iQP5x)P_F2DIR=kGUl%3Qt>)9+WIH{{GZ3s>wq4n*qsa9mV% zuDaaDk#Es<^t>7NN4p>ItT33@(fe2>w-!D~%FDeQMuXKP_glaVk(bHmYu^d{)?puK zoP=JFYj~G4d)T0gJm>;KXGFghKf2wgU7z>&a)(cVyB4+3SoVm>ps{)Jm<#`xH=iMg zQ230Bw+2=5sM~z2}^MucF^?zZ&kh&{DY{Fz36WZO_p6S$5xL4bmEf?ndW(O+5Z; z`oO-~r(^nl?)(mypMTEJy|-5SSSk5#HYc9<%y)k|XICR_6#G*K^8$hP9XP?-Lw&?jmQtb6pob^W7r-kDU4b zjljgs*lnD1=6n6O&wu9o-AF&8XSRP%9Ute+_brj}2z``sRFjiI#=n62!q3PxXxHs& zm?LBS5Y5mRr5V3lHA?GVM7}fNSJAx>Jtk(RD3%^0v!gF!Q~lt+1NE}^qGi8UW8YEJ z0DkpbB~EhTmOoWIum44aW|p|Z=OW2}C-kxa9F}tpy_+0?I-!HBxw}}#gsqTR75`5e(;v@pM%E!~ zAIE+hy=%FYZxc61X!GW2{@y&=%ey8_n-7HZX&!$}@-~HYAM`$^1NQz^srLEH{Ft?Q z3!1ei-uDPL(9>mN`(lm$9Xk5$Gb;zbL;OS=di5HcKBYFune+oZ!!FnKEU}bAqbAyT z4W46n6PfK&`?sgzkJNE)+4OVDj^p>;^d-6$uL|!t&v@U4mWZ4tcq{iEL*K4)k{=(N z_36dXkP_nG1jpnqT%mFAkV~i-IwJ9e(8~tN=RF&nJh_D=ALY_Sa;q(poWhj9-M;fe z@(Cw3$Qg)CXw8M*=b-aRyOXiO7FRO(4)u@w;8CmXSz@&}-;Hn9vA&LVZZqz&&(Dj^ zU+8=b@wz`@{-e(~(_i~sJ$<@%D&t`eqqOUt72F^02_?|4 z3Z4mdSlvbbUjE9SMlM_LfhNovTe(B*ZH3@H@m80}JW6ihRd$X??jZm#>Z7agVaDTA58Qc(JFU3t5~;lr}0)ZhRz2>`SHLeYoDR8o0aUR|Ayg94o#*Ps7?X za?gq9gsh8g!{#|>^@*pVy~w*=_`y7e`zo2YiOiqqtOK;MoV-qzvi^EsA^UuN7wccn zdLxrQlEwViaW8`Xe)BVhLC2g7^x2s*7x%FD53%mSiT`?59!h!s70%eaz`HtZWpDO} zw!I|fWIXWXFb!R6;}^8T#v>{Bt0_XG!}o6q4h^onmi#r6_cM%3&|LKE3I0DJ|4H6d z>A$SYN7Y+Py)x##;M#ZK1HwoD3Y^LB+xUpniC+L6@;K{ooyOk~M-DljiM`+^$7~2M}j_}Y}{P{d}%U+0;QMw}e8<0=Vm%m}JwJOy=7dbuJ z&N%i_6wZx-Uk9(sB`?h#)9jqUlj%dO?kMfl(}qe0NN%RN;rtLH z^N2nsV+lPcc>%-r8R8oQ_8QJm_uBg9?a=WpqdKZG$W+~!>k`OWLHo8Wc8*>t_#*w`KE|_hsa)(E_Tt_-4$T5?vZ+)llcnYU1+a=o-Ln_ zuD{G#IDf}pYxCsu=I@BE&rhi@iVybsmtF2ZKt7_U&V4?3xqmayTcrIh7tHgxcaECk zzw+oulqsL$pK6zJFpgw$F=MACPb748ueIxHzccIp!`9lhgZSI}7wJp=$HwWvLVgF4 zGwR{fL5Zv05*_ncaz)15IZVRyn9T2lKyWU+^wUt3cRS)6 zh&8h1{b>F(Su5gRV(OGNN1w3wRkFXAx@BM4^Kf8VWZewpen+65IggdCu%}b^D1ElS z$(?>)+KjHSH@9rMjmHyW*7a*Kbmnhj;K~>{mj3)C=6T*rvc{wPR`7X^6uUI?#EcZ@2^WLbT(h3w={^;cuS3 z&^P2?x==g#z&^ac!#KG2Jo$O5tpnt80te5{&lBC>yFO`sVPuFA7+rS+=8V7yjLtR!BTtXO znKAICG4R9~_){_P z&WDVY`DzS&YYhCg82I)W_!}|sw_@P$#K7N;fxj06KM(`|j~Mvj82E=V@FOwsPh#L5 zG4TJ2fqxbQ|2zh6h=HGofqxYP|0V{0ItKoS82H&3_zyAg3o-DEG4RVV@GCL!zsA7- z9s|D~1OI0Xd^`p|83Vr=1HTgkzZV0aiGhuh(RGiHffHlkF){EZF>qQ8oEZaO8Us&^ zfj<=kPmX~<69a!X2A&lIe=Y|8d<;A%2A&^=|DJt)#p_r4`xiVDxJ&kUe0aP|l=4b$ zV8uUyB{pe71bz#6Rs==|Ba|#$M)EYQi@?W$ABw;dxBkB(u$)(aDgt){|9b@fci`a& z{3>uJg08fC6qs{tHvCV(H$>oTe>L3~f&U)(OA+`9;M*eb#a5ZpI1pWc=4H5WZ;J=E% z4+7s2fhA||eG!7Xq1Ait0mjPcFfhDKIUq@icSMXp2UIF~`2)rElxd?nC@M{tH z24H*wc3am0dkMCZc1wXTkHGVQ=SN_PTmO0lo()_Xfs29vX9O++{%r)F1^h|`o&kJ1 z0#64{!LTK5O#_}9fhPms5P>B&DZs$E-P@DW|X_Bm=;K|^;LTsavncZHs! zHY&Bzwck*6+D*~@It%!yUanN5VKp1N%`lG`x_$I3slUb0`;6&k&3HAllbWjycU6|^ zGh9gM^$u6tc-8K3aRYr_oYgg6HN{y20qX+^myti=&^dntFBb*Xol)bb+t@TgQ>SDbtKK|$xxkXGwF7^ zw{e1MO!rnyP;Kd1gIOw=?%JNE%Dt{5bkdtup2a1m0=IdyS~69?H>*EGZO?FZWvKcL zS7(N5w&6A#-aX!`&ep*!YamPSolvQ_XX}4|JzA5D<;!|0ov%w!t=hXiLDd`X+5|Od zOm9h0d%@WRb;RMRP5@`Ul?kdY-l`zdC*IY>lXxobw86f3PgQ~{Pq+)sH^CZC&|yTi zak?X3KObMEUDcVY&9u78RM50K%2aclHGG2_j`LK4KM9_87CXVyhPhYfPPHYv8Z*@X zBsXi`=yJ7Ws9~3j`{j0zakp{PSc+8(oRY(W?suDgUe)4ey6VT?!D^R#ykJqS$87Ve z-5zVdSMBwfd%0rQ1L_1luI-s>(Bm4;P?f0yZ%-BEXi0T7WvHX6uDT2rOwBsN+cbI7 znC2Srs{Lv5{79Os$E$kM+{0cqn3h!uN~ODi>(ix7bGp3kNOyI3Rabh}Q4%bAvyOOG zomZYTddnGAW2UQWxo*w`oomLs>&NTLOI;Ots^-#qUhdEF3{O-&Ic9a98p!cf=BeRa z_aILuy0_=4raaf+d=<=d?VhjNCr$5Ppc*f;_Rm+91+K~ks-?hlWWK7JY#yDjswa2A z7rLfpwai!b)6FX0PIq<9SDn+Rx6W4`S9t2?tNovK4RgS4hFMFQnO4<&RWZv|zCe{< zWz{ZFomaU!=cyxw?jA}OHEudeo10~cGtd8)5?`tAj)?i$y2YWTdj za=vPtEj3j7%<}naz-JB4Q>`WD@H`As*1$Z~HOJLIPc_eV_smo6b6s8YROei&r~O)2 z-#pcSZBG4sRX5LFUaFerc`6v`JQscLT441p(9NZ;BMVhusoAkm4VRkj3swC>PuoIu zbfK&1deyqfYPnw3EVlMvuWGON0JmOmbzQHITyHVuH8%*d)!tAk`{WB=EO2!zY=)C=lrL8Me%a=UOE7bNCu67=;u$oqo z^ucPmNq634ww0-_n}UL}H@jNPRQWAdOPQ+psv!FAuL`2~eAV1vraD$y&6Hf}YAjR3 zE2r))Q~kHPj;vFAS6dzHRPZ)e(>hi6*RH+m)RC`QJ?m7TERpV9&6T6*iz?{U<`|5b_UMKwD+f2!X zNqm}+wcV+D60(LZRb^!jkp0JU^;oLak_wL4aF+!i-0!sZC+aq*)#cP3B;8j*=NcXy zO)PFnQvFF*U9zfnxppV3{VuaMS+%>&?a6A;+O3YHNSA^(Ty=@H zdmJLAw`rVe_lS(y=|RS9OI_RtnC5BCR88sbu1wgp3-PqaYayQ2Wx6X6c{42}h@<0? zKqDf|U+C;#LR*9hQ95Wy5_!Vj-?=iTa4s z>T~K&A+ZC_H9Y7^ysbS+9Z9wZ$Dq17JXR7h+^+Tz{g)!uY>?Ra%GU91F+UQhXWRhwZo zqS9x&up#tkTEm&DI?JlfifW0Ul*VSdtu<*0yxEQ0hJ6d#2oIYyi6BQR{RmbVdxR`06??w7b zw~4lzZ+2*AqDL#A)uh#isc0CxH3K=KtuC!Gatvr)VR&~Nx^~pcWY7C_6GdyLRf}@M z^eib>VyM`z!FFq})>s=x2lCPLF+qE0ns=$fYsKExG+N1MucggN9+~dx)USN`tooX} zFq~<6PZzO|>6+0lGR6$;@lGYB&b6#!qqg>I-4xTQ@EYHw@|Vb3PS9!09XgibLQO}p z+HdHlnA)UFjn>r0(%12M@{zNmFtXdMqP|)IXl~dehBK{1!%<5#dJNdTy6pf*+ZH(?1V*&S!G5f~o z0UH(}C`E5ik>_?hZH}BihaSM(m7pt4t0h5K zo77(s*QtSn@z(AHU6tSgZb@);hVKWrAbC-zW25vR#jH4-P(3BT7aTYIX5KX3h*Li5ce;T)5{p}r&y%cyF=}s;)tIb0ldO7BHpv2&Yh7Mx%Qcmf11|Zh zI@!bX)?}+cMfW9}{bTg*F{OMvJmxN*u&7atkv&zUFVDKGT05%gzA2>Rs0ry{@oJ4u z=$9@xt^KCnZF+!@ny#RE*5ZYB!e8If#a3OGUgFuE1qE`$eQL3;XJKeHv32+9LTugp z4S_q2!di^Rj>7iwigY!U0S)CzoM$+TY4NmYsrGmRi($B)iVW3mEve5^T~7J-NTLS@ z7)(4Oo7m;8&2XTHV5{8gZZ{~@JgyK^VgEQ6aI2>fV1IjLuS(X$3P?OQr>* ztr)-C;PqKEDRa42-iN)q?AN+SPjy#j=$i@$u>YwJ=Nmd-KieFZAtY2V`JudpAA>0mFs~FU6pQDd3AewF&}UD zw#uWkRxq^lO7?=MRrni7wgi5qHkfrXbrrqX;-HT5s1eFT--1 zIZ1guO66OHdU|XhkR9A6KD5OMTuYD$!1_)R|FTAEmwuKMOBa1U@DOp$0jWS@VP20K z(9Xp|aKt;)Xn2vfT4iz>Zofe%z4&FCWBobT=vjqQS1}cAaN*3U)b2)msu*mm)KzVG z2Mx8?@G)Qg29}ZI^?GFy74sr19Mp3|9WO#>h6iR>k3~SKP z^$ssn*Am;Gg?etj>xe_$;^6~kFK93Y_@WpcJ*+U~U5&vo=#i_&0Wn0%LIbH1KUFC? zzq&?ilfK#YbxM8P9kP)9Qj^8fc1V3yhKqN#2Hk*~xN05D7WGx)MUo{&0MNm+>jO8l ztWq@zVRW%|izeHC)WH4c;pSMKJRK?hHWxDWT_8l5Ac;)<@o*No&~28JY6D!s5o?Qe zv3>d?w)L4x&2&{cl+VRHF3wVFO%?)pg=P$m+Kaf2cEJOv68LjO7*?BM9Wiv2^m%tX zbhSg=g4;)3dd%(r$-3=%zDX6LxZGl`(YJwjQCf}Ol4TXENeoNm-Vs>*3rnmjL)~Q6 z8EjpvQ{!epg{n4iO+#v8>JXi!)ItK@=&IM?{B$GH8GXKsb)>9&w=MNYUr`VG6@&QN!tSKc+C(W2=8pti7y zqMb)S`?zJi%BPf1==O5v4TOO992xDZ%=ct=4$x>Lz%l*;5`-7coj+pw3W%ml1;FOk0EmsX3}OTKo@u6iRS4GVCuF>KUZsxgi_ z+u~e+M{IGgKmIOYq`NLlAGKn}w@}Z^w8VlmGy8e z&gSazS)~;Q@gf#DSFg)#?AUJT>d{hyL&@4IoO*}8!@MWTl|Hd81v(OIFB!Fqe3b3u z|F-Sn{{!2C*ng&mO*;q*&4$$~0?^bb)(Z_DnXgz_Xr3GT?X_;hW`r4K1IrwxmC$Op z1>R1=4u-J{%-U;{Rj4r3uSG+LqV!&9FlBvFtik$Y=-meKQyQiM+=@CV+JlJ@ zTVn@1ZFh`phzKrRSpNSgne8&2?Ka_29@!gNcU{yLX^-s!y2O?G3U!N8w`|8kr3N%$ zhv6P{s4nALL~_R@K%`sV4LiKGrmitZP$RSxarCGyeoKOCc6jzDsD7dY z6RI`PN>7H|bfOdJ+FGak+@-?qmgXb!|pYD-Yt<9!%8CXro(Mm{}ga(J2(6v;)Yy}~qk0{0`z ztA%K-c)Sl1@x^AMd}Ex|60ch0rJ)LwX`Y9deJ( z;eET|V&0D$A|UlROr8WCbSAtPz1FoVhhMf#h~!{P55p*B#vpLHiHyKny6nY@nWix9 zIZAA#gLSpg&TxB2H!buzsm|d!ivHy&4YxDKMhjObMrSZgo*@s9FBVn33KdBt*gNc| zq4br8FItn`M>m8TX+ygh(teZ+k@3#=Y%JoVEkdFg9AM#;V(t`#3yT;#-$<8rIci zs)la6f+`oM-P~A_kN9_12Rc_>HO|9`7MEQ{ianO zrw7maPV5WQA=89%Hfvq|A^XB&H?@99)v)VML(fqB7mEngi~w6S-_a!wuP1bnJ@0A7 z1mX}MlIWw&@m4*1J>J9aIw~7zdxBWG>k_g?z7DV5Vl@dw3&YFRB*Br3>Oau14K5II zJEjM5wpgR}@!f@WDrAU{D{F`JSgNPX+Rd=*WAKr1Mdr9CLeqj864oDg7j!Rv5>&1t zp%AnU&q*Hv)`X1;ZAr)pkI&cL10 z9G4F);!8Y%%JQ55RmfONJnExrNwDx3v?YwL3UfI@)e)p!mteKIuy1ZRDVAilx>Q?I zz1rlJ~W_FGA7Wq&5{2J z`+xSFCWi4;a82rgUPg%ZB&?nNdJ;B1JWpbm9YsOPL6?S!@^`SZO?-+SvE89tV-d<; z=ZlSB=m_*y@!#Dq_SGH+h!?HG)Wb0qfg6O#Q!I$z!-3yJcnR%+%dPg<<-JJqJvygF z`Xh6**WrT6?GMk*$J8Cqo|g~fn&>{aSZwIqq|-gN&8KfvgchIAXCc#UOf1DPUrfC$ zk{CC!BtkmSoUu$+NS_)VZj7(QxsOn{@EW2A?G-%;!Fy^YUpHwBrvVHL*}Esk;cxqZ z-5xgUB*@eGX!}R$s^W8Yu}cb32h<%V+vlz|QJ-^_;XY@|?Rt1sW>jilQze76Bw-_m;O3O=5nE)e&+3$AVsn=lD!(y&ywj^?QF}bI*p%REP0$?)E=0#* zf;Et!hZ9^?maeg^-IlH$HC5UBEo*UMomgO*TsF1~n3mf_tsdQQp`CSF|5P`EbBSzE zt0_@+A;sX!a(KJ&cR8fn9j2?#)^n>8VJgJSz(Tx7ol*X5k`${h!9~=}-UL^35}YNU zC)JikV9s94L%>d}#R4=rtroF=P46I4W}?-Uq<1G;!0m~*0q;+;+LClflGT}{gGpsP zZ*z@ylkrT$k_Jb6NL;H<3Dh3S^@&*m+ZgIGeVqr5UHJa!$;wEl2ISQjY&72R(jL@7J z5t;)5Mv@bsNV%~D7;_!ua!^jqItMiR5Mj2e4 zrd5l_=cv{kHi#~vSER~jYc`uHae_k@l07A4&642xWX$Z$jQ1T zZl>cdYE>JM`rGxSK{2qLRsYn8T;=W;7nb-ATg71CWY{53##GpekyYu?<)hU{ah|cG zKEPlr|eSy10doS}_E0$-qk=Zp#c2>@@If)T5+36h)@V>{fgvU)LV4Eo=+D&?0 z8z%+o;^?*-j5n)YYB;_QxYjZ|cy6%;nyp)!ToU8lpQQSn^0*>#5(W1sW>vV<(ZuO2 zev&+{P4ZHt*@iol%5{%RcgB(h`Y3qH;Q4Q%1Z{Vipm?nVWKwmeizkh;=UPp9+-2re z60S39$&)eKIw_01M3_j6t--N|7_mBKxuTo(f3x>B@J*fd{?9o%Nlr^>0|e+;VTY}Q z!b+Q#(uPt>fx_CYKnG=;j@hP7+GeClm!wekau>F$y{XlrX`2G_GVi^ge;|F%^PK1X<@f%3Lh|h$Dbhg$ zQ=|?9fB+UTLx%b@R}Mqx1rZH2NPQy7ykTAj11sz_?(T+N0rF~4z!n8qfu7|g*9mtT z(mEw!#Naq22?t^X++1&)A>GdGH~?PxJ|Prc5Z>2H2~>!e_5^A^&h5>B@i;jZ?W->P zQ6Lm;6(9ix4gkHS<3-RTQF$E3?;s+#oPb*|7morDf{#5he*79Q2-Y27Zvrm}p1>vn z|8hBNxrD>e$*spAZH4mUxFAie4)YH9Fs1(ed^mr#ke?x)AsON+=s%u0l9H57N#BaChhbI@6SzlP#;k~O|D=W zfN>$Rx4bV>1BYpaB;_au_+r&0ENyTQ)<{7bP%P-KGeSlu*V#l?hW6vR>6w}7ay7G; zPX+I=uEpG>f%u-t8uMo@{YmH(e=oENwxe?a5p6wl=s?3GU|6z}eXUUr&tfAc>)=dw z$ZQ>%35m9~cP49>ZJl#iPl9!D29B!SGlO+av$f3tcL>6mg_AA&XR>`TlSN9h)H@4) zu7I)!t(N1nVeVrYw!v7$vfsuAtrl3)4qI(!t*m`U<YQ|Oy156l#f&y*r7tD2B$!*!5Y=fouWhhFq(dn~hjxO~5P%+|pinw$V`2CX+gN8JrF11?OSVocNAlpbQIs5*Zt1tNk?As0&Q7;> z+gMvt2f?@G+p%P4lL4R4SRF7q?we79gw~l>WbB%min>nDR8HDh`z*_VjUAYkiq7=T zvb5RQz$^=%hh|w|z<@~}-f#h=gEB9k%pPDwe8?7LC51A}y zakHWcge=K=Q01^mJ|znyX62wPbjr$rEDXs!B}&wiEx;}1@-S40LfU>bA&`!yAB}>j zG1wspI{}jl6VOshFMbO{vUGt7mr|hZ=#gxA+ifiGg-F;4?9yeD@rd^IqsMuyN(t0W zSWFo-us+DF44~0vP&&uOT;L-^Dv}QwO0bd!D~O)66L>7}tyYk^c3Ad0XkLW$-tp$M z*nbzuFUHk1VjrfG4&grHt6;%MR7pDGV0lJFp`GAdkC=wXq4E2$QSda_1|by}ET>Hv zfP`ltd+;70IUo~)GWQ@B3MOVqH>P26P8nr*EuS&s2oJ*gQ^wj@z>*v?D==goHOsgp z4wi0s?vcsx^|dCYGw^mw*atY6 zE(5iet9U;(9rn{e;$)Kza;y+j8Y-T!Xq=|!bbl@uw-A=>=QNk+ZAauyko_!xr#11y zhXv3Fpru?bMD1GvgkuCltRRq$#Z4px2b20Ex6eMZ++jOQh#@H9- zC(9Magn2APsOgLZ!JxI@i1S!+h?JWE88ioE7l977kjlReazxM+7913Gtq)RVaX&Z% zFb!!v;7tq-07McQo1T=UGm>yB%6V`)JROs72Oey}dzjeMa~^yii=Sejfun8b zkqy{NiEk*sJ?@^M^9HgR9Vd4KKzd{YSJzLL&oLdObD0X-f%tQh(PPE|yuo)IB^_~O zTVuYi7Zzf2AfUxGiBQmBPKG(+08s}e9k@1q*#A&o=uY;*r;zBE6h2>oW;QjSiSIhfR@A@;$#=a9GK!CygX*g+yny3N-vqfw10yK(-+uFxnxm01FKb zrJ;PkBpfG%tn)QA7j}{!ln~JE;sTn(F>^uhD~LO3f>j-{^ALfaig}N@Qfw_OSD!1K zcgRhFZqon|0NGIT6qzr=YlbWr4~WGvZQm{I1XU`**ml&wN~EI_bO(T<6*6YaMXC}^ zC4>L~E{7pM=2JF^$tAfmS&D)=9EGV8{M#VJ;!~zdXpT$rC~m`%Y+;%{N2^KbH%5od zX^MF^a1+2WBZOnnM@U(D?+PQmBUlePMZ1A?ZAT6A2?KO~5eTU2bXu`NT7?You-#;w z31iw5q`J!hz7k=$2^o5%0AH_j@o&&$7m`hLG4-jMOeEXLL>W6M1*yF>SP$cUDyo&D$F%0%=-ijwj?%b zC#=f^BHth#Ae&x0aG54EC;K;;nxaUj0aOF%xmF1zNHGLr15xBgV~xd0c#h)f3xfsc z#$kY2RtTK2$FL{ikjM8u))V_4lh;MkoCDC47=%NxG{m}zJ!!NGQXosaxPgHWE%44ycP((Tj zQv(zP43J>dN81+y%0v88ovv}RjXQ-~vHC>0f_$Xqfe`t)a37hBIJnH2T<7<}{VtJg zDje ztcb(b@t{m3J(3T@VIPW6gY^{gC|0Bl!@B@++Y<${~a07x})czz8zEZyh zQaAb3G;F=X;}%GFDFuX8YZ48=8*cr^lI|o}kWfmmDWfk5jz@AkocqnK0wUoA`6P-Z zq#a6z3rEUe5<8rjax{qzB$h+mF)a;Q2B)RKhpxkt)<>dr89!1wNa}C6cBF-qgtjC~ zIgn)ON)pZ{r5#KbI+Nv*WT8Jf1?fY{*68-)IA*3bB?IzE7|K*j5a_1hK^rCQKccfn zQ8(yQKvF%&z5c*cA+%JdGIHlSJ-Z+06dc|S$44m z-A4(XFBd_|c7a?t4+|+s<*0HW*Yz9{Gfs*)Uoo)g955~xCU6{lF5lJ5L6>m9ks-1n zJ4H5RoutVrAB6K6LqL%G%SlG+wU*9*NS!_guq!T_Zk@gYp1zfL? zfeKuJnuL3SV}W~tV*z}Cje&c?jso`%kkY3b_=S`3Bb*UcaRJ~MKp%{2I*h_`$#DQ4 zpvD-mi@kNHa9g>w&kSFHlP1sycseOq22C)78zF!tNgWVuz!(6(US+fZ)Pzm8qmZ7M zdcZUbr_GR!2<@_rX8@HGve3_;2jrN;H}+&ZsWntC?I;Y?VG;$cA?6JXIj9gApNTHO z9RuA|)?*+GISc{p`zZrW@EJ)#p$;Q)@tsDJ)BuJfWyGjJ)!1fA86{Z^LBYc&;MjfW z5*ekDiQH*(1{cL-49Vzi%rVBe*9rwGmQ$EM;3xwm3#E7-5$wPNy`UAy@GJ#0e@09J z&fRAyK?-KHO%foJ8MhBTLhTM2*0$IOCrF+nLt@h_w%{4&e+Nl=W5F{d`1@ey2d94{ z=F~D`1f)zE1e!9Yz#sRdQC59$+fDGn%?QI6)?^2*Y?ASKHY(vESOC7N&J9j6)wU)q{F*P8_5*D#7Z4?=I~iZF8g;e_ttK4ZSHS0LBzj z_ZVg4`QO7g4npx>5kQ%&PFuy4lQ41@$rEQ#v_MV-c_o}}+6-_=g2)rY&;bOuzrBRA z*$jZUI2VVkTd+fDHX_)-4IC0F@pV2oEVZ4})`Dzwt%n7s&~KaOHgO>bj2Lv$M09#EKbJyt zlb#dJ;7b}u&}Bp8J#uKjZW78eY~ad@E6FYdo{1zZLi_}G@gseEN2T8H7;JJ;CBA^@y33wSsW|ioT zc{@no(_+N|Y9n@nXi4fbTGqEQr`r|C`_%gi&b>FzjnhDvL>yJ!&x7pI;!r6dF&jZ15eLQ z?=_gif^^tmY!%QN=$R6v^co~CB_T>^$WW*@5Gz>cFlw!^it~gL_`phl8W1os0~=#zw=DD);6;S_E;2+^p~2pz;p$$XLk0T9ff(Y&A9 zhRorNWF9iKqXjTKr(rV!4;mt?2=VVXQWm3h!YG_HT5tiy8KVqw2Q9?-4y&@&PE>+P zJWGNcYqy9l7n9ust!<2SmvFvP313G;h!n;xglvHjAs>MZ2XylRvaBEHRZinrh;0UF z+XT}vLs+QBz~LiI91luIKj2-`djuEsCI>{I3lLbvcs^+&=Yf4@?DkGGncE&V10f(U zRGuIT+c}8bPt7828nDP&KvEGiO7VnJ^4=sB2gPY`mRdTHP#T|B#`|>m3EKhW95^)P zDPF-9MGBloNGH{X9ea>ymtoTCM>3#2$X0=F)2J<+ky1JbAT=QU?u=3X^LMwCdB9v&4mN2SKI(({12*gzrP;9_Lf(scW9G(S+`dy%z&IU8MM?Z$Ju4Zr+3oNgId&W4uQPUf)gOxvj()}Psh zzMjrbJG-29C3iS&&a4I#1G!^L!VYHi}>18%>rvtSoHJ)ym>38Ql0NoyM&; z$p9$4KFFKc{v-*)q>dy#UnhK}JEu#yqo!-Ro`xHh*um*iFI}mmr=3N8XOkt|AUVqG zBYVm-1`9AlShCRqz@-?{zUT=;n-`LY<^ymWUpwZlQ-g(bU?M?g5_Yf&2MielB&p_r z5aU>Y8KRTiEwm*}awJ(a|qu>>nrln`7| zcA6W+n^+2=5sFiJg{x`F!0I$P<^dxEN~?ea`7PZrMK^(YIgIT>_ebr5CD~9uEI0;) zsVSbkjDly zEXP-{-YoCg6|5)6+K1mocgXDyq0Ipp+wGA1972yH1INW7hgS-x94UH-)G=HKn@}7I zc*fxlQON=!>v(6z#M{$nFtpmI-w`YUX^ZLhqK>7Bc$oD z*?xr9 zoQ*tc5Kh9-L?UYpn6pA+1Nk!~&q(Ql$0@`G)MyOs*Bx{S7WN9z1!*totdtP*<4wa( z+b7UALE3DF9aPc|_5)L4ZG}QYU9FM`VN#OecgRqk6+mwQCTHr}Sa7vc56S1c zsV;5{iUGjfs$oBjVT>tQjuXZZ5=`<5+$L|LliwxFy|QpPx*BL_hL^5J85C@ogb{&Y zq*gyG2AgedVHGNTo`D{4JPWd>UpN@aHv^IhQsSh+3LtC%N2`^e0g{O*JsG_o6$obG zL-T-XC;Jr=lpN4t+B6X8gQUL!XrMgPSv3Pj14t*MHcBIUUMWQ_hlq%W-l72jBA$Q8 zWzI+Ipu)_QR#HV`LPtb;gfl7i3wHVKf^Crac=QMsZ)b?;^YT8^IB>!XN%dX|grntW zE;3m@4&e^uLQv*)7%)!tofR_+c@GyI4{0zv72BWGQ>02cE1bqNe)JX3qXsmn+MuIN zNW(1XRU6z%4Y=oI9L$`k|Hm4*XoGg4PD1n8$g~f@1CehTnXVm-S>HrEIUw@I;Q1!T zQ?-3zgXC%uR1LvcE96z&B%If)?wf9Cmccb>>p3PY*6CG1+x;S@2e-5Ik;<$X=ZcOS z*a4y|`JO4aVbS{q3%1lCnX#kZgi;c>?7&&p@_?>j_*(bV0mDWjo2y*N24I#1u^B)s zd~YDR>?c+sEbTZ$PPT;p&zfLuCWOgTt=$YC+sJt)`xaU&+mL~^pk?&ldX!dPTQA)2 z<<@-O(_or-)M!0069CeJ=hMc6IE(uda@%LH{sbIsxVTyHe0!BI2UrgjLVBZMwtOV`Sx>d~=eG&~bwW z)6g3c&$CMNZ#C2BPc45T?3%SKJ2^%K%E4+Yc({b^6+Y0NTFa|6D_A#Lt99- z^i8w$6tMwrWO{VE)Qacn2atX;iJng-;cN{iTlW{T;pAdWmQ@}oWSv&p@O?9+Q>$42 z3=kZwZJu>x73-KcOh+}GAY@+sRdIaDnyAD%O=|9bUzH(}2?1iN$i; zYBs#M1&`@cUlAKh??%1tOKpRN?BLQc>N=QVJy^uLGf>xw3`=Jr8_B>1?pY=sTFs6w zlMk$BryKDYoLVtJoU5#$&=v*xmswfnEWy+{gWQIgCwW zUUCxvnSk|J@()x)@HW#-VDt#%wgFHi?wYtJ4v3atlxSZzW~BIW@2Zv zjyZ>gLuIV(Dtk{k8@WpEC}rLAtot_M?nD6n-m9&HKEO(!RjjeiJ*e0GniJLF$=O7QV)Eyp)|#EJ*HO*e-P5$Of-%!nV22fve=_1mOaCM>xI+&9C--F>~rNqDZ;6_h0)4^PshnixaIKp`fb@^L##0vE%g zbOQPz5-q|Q`4ovLsVgG}yWSO)hX8C4rj?*t)cUSsIPVS;`IXX<4T~9|vb)3*wQl4; zARLEs3)XVz6qRok==WgugB|2?xl(CiA@v>5I1*tWG63gDPN-A=XdDFPIt1%kGQ`0d zL)wfTWZVEl%|0UP2k>6ci>{D#Vm~Q6)p{2~-v)Gn%l?3`$WiXBn2yJjhI9y2_Yr48 zWuHtCqzu`i!|kl?L3bdSb(cF3*prnRXby&&o3cWFzqhu*>G1`#vNKoOGwsWq&OmKL zUhXnyQV-?z&h2xSI> zYm*w&V zcRGE(W%kV66`6Kkras_oYRE!?P*zjGzsuut2d-FKSG)mrF!kPM_Z1t1+JrjOlh))i z?t@x?z|9w^t~u0<$AH`E+B11nm$Aj@&I@`ywU=oI)!qk}HHSQ2Y`f)|%d_9zQAOs_ z6LJT#LII~I6wIP$XUNZYIvpGwzR6R2F$XN<-$~~#Gk2wSyaLU>kf#wVS=-=tHG9#8 z+U7vO?F&_f8fZzg8vU+!t?9;QZ^(1bhA-oGicCD;O<6&IAe0q!d)>8H4wm`cp8AFw ze<17bX7}X+0eAU4*ta>^nd-jX4RG0nRrJ@~g^>iaoXsJB6-L#FB@JGlg`5&d2i=5C z00}U)mub^XF*+o>eY++P=Q3`OV93P*L2!?+Hmk|+3Hp6meqTJWb43~rX7Prp#i@47 z6=+vG^T6;+cUSFl*T#o4S73-9fBdE{2AWQ>JI9^9DdzHGn!5ntJr06S z0c9@Zq()%qRLJYnNy2}0(C;R`x>Bez4umE8B&b34swU9!laJ@cY^gXcsRX3cb(gc& z?W^761^Ir3+q&|^Y82c|K{{T>Gx1MqD-`hf>MzqmIfD(2&ZcE~nb{hZ7?=LMjC1x6 z!ja3^a&)y+c9sCyV;*>;tWfZRxBKQA4F$Em%pF%C|-j4Kn_23L+QX>ta3 z0^(MA{h=yq4ouj}%tig#DMjLStj&BAknT;?#i^B;S({jGEFWcdaTf3*RsS3d7GA;Z>c_;Bv5U2Wfx?NS5kG1(n znN&g$S7?B`K)2Bij-WP}wc8)q84m7U$yFz2pE9zW1WagbR^JUDl@gG!iPl zLbIq3u?EBt(SKRg;1z0%ub9st^3+`>N-O^P{ij32sX){ffFJJym865JE3QBD`u&Z7 z82>~SoyX;U*Unk6p*iFN9e;(+8pmY824}!EC1G#{w&S_=EteHA0%3;SEg@aKdxfS^ z2akGdTKo#PHU++!3a(t1dDRe&zv(h^k`swleKsS==JE;tC!vB#ZRwLbdF`&iK?v>% zW?fk+!;}No$N4|p^5L>S|g@NnsTRQD($E z5^q9IG7<@T8ZVQ~7OVmTs&L^VXX0E9`24O*nw)6eQLBwh?&d^La*-8AHomX9F^d~r>gE?z zp?V%wJ1{zs`8rbI7PZdW20zUE8f!Ewp7WLq+Y=#(#=Qn>{Y`F;d#1<`LuF2pBVr^s#e2=Ds}ZgX zlX}6WWTcHG{UjH~O{}EGS-X?8baBBTaUP&=JpL-x4%p)dl8raoC=oQVe!#tkcqMqx zCYX*+EUN>MiRpar$E!3mGYkJCL*lfCx|clnxOHB1Vr?|YCg^Ul>MdD*PWGHL3uubk zomH5LK%LjWTj$I?zB+%rE{&^we9`k~C-YiOn15lJ8=E1=IB!;UZ=OG!2EXIYMqQB4 z*Hm?o+5-YO?m0769=-##(Z#AN;x)(fj<@%nGFxCb?e(Z(@RS)fo?0_V$Wk4Art0lD z2HXXYn?N&>n?aBestw@8X5#I|=*~p$W*_{C;(~^FMWfr8Q!-iJKy$pVUO>@{ESA+s zp!s48W4ysC)S#X(4uCI$l#C!2IBLO!$sKC&yHq|UiYDR{dd`}ffLk?+zf`2=2Q?WB z`D3sNuX>rw-RShy!(|Ls6+k|!TG(Chtc?;ADpHv``z||8^o7-|xxYkbU>wV}A~Sa) z%cbwsCRM0lj(+@lL989P=>g zEzG+nVxmaxDwb8fZLq`PLCEcYbV7GhT^L|2V-feg*B=b>jTa#fBFzEg)%sm-_zrq} zoUV>N-H}{0=QW;s1oYa~+yrVI)2et}LB-=!$fd^TTBdT}YDCd}d4fvFuPD1cUa#VA zsdc+uN~poDG&)-pm%AR!8pTj=jXR+D>tafIg34~^9u)Q~4e$zzwi&J1EogX=rCM6|Ni-2 z@AuVt>Nk)jv&ZLi2R0LE3ONIz3O|rR^eMR6AKLEoz+GyS2grzQHeDs2#vs1y;f?MO zt#iNE9q?~*wv=#?9-Fw`x6=o$w6c}JAIFM>FiPOZzIO9w<)+PLcR6=CVJr(vk*%t$ z?FL|pCW2PD2SEL7bb1Jy0UFf;3KQBPKY744>VAPT>9y{rK6MVR>{2wXM1d0RLmA*^ zz0-@ahuo2l=)Ki_^%ZXXJ*ine8}HquD%S?;o5{&EQfrxSm(vSSuMS4lR_JLO&Zudw zgFZ-4L|!!^+7O{DrB0n5LJ!DN5Sd&R1F4e^b3Tq?IBKEEBmM|*pWHSH5rnFofq-*Q z#1mHGQT$aqt#`retZ{pTddvkZLb7#i1a&!82Uo2_(T-@zDox~e>BS?-0cT^d04PtL z4{Wo*9t{-+x48oz%+ft>r2&`?$V?qZA_g5ZuB8H{5*v%iBbVYrKlO1nI_o{P8mXhF z(R;>laUuWUP7walYWO#v3FxHD5U?1ZW=K0oM6+=3eimJxx<|X{791Bi9&`a3MRSPr|x5yT7f!m zw{oS-R+ThzdNEzK0S`an=}V|S2;txoP4l%%X%4z+|2BD@dpKM|TV$0H^fw1;-Q}>a zM=qQ`eJ0fK$dROn+fbd-lqgPfz>9f&KpBmtR=;w}l_9`fAIopL;x! zWk3D%zmNT}La6>_k^lFr|Fp^Gb*&SYW&G_LoCUr|L?$trSy>}aNfA+v5DSDnw0o+Y zp(=IF)xQcTS9B$Ssd4&r`V!St&>ZRu3N)|Y zf`V}_^%_O&^wRi9KB?2gm}4DsY3Ha3d(5pk>oK=b-eA_r#+^C{d{avREIL8}eSM7M zJAoJ+-(mShwuee{Q%sp?5n!DSzFkYzKrNxooC0Kq{a6Q~u`2^pgqu5jRFMks^(J|L zD49SmF*STM>%GO*k4aC<9Bd;p7AlCKiU%7#ezo~R-==;d+z>}VEIG*M0vNm z!Ai%Qu>a1bn48 zA8#clRHf~)_^XDz>(B=8>{i^=ts=4%F53imzi zhbGSc-WKrt>TkN$iCc3j_cXaR+_D~$Nx)oeU@ZXz2(f*exWo(;-9`>NbNxck65>L+eist9-*0gQLKxA3bw_?oB5FeLRXKovjk1q!G& z6n6ou0op^{2(2TKpUYhUVnU@=wt)IkKm>){T2(wyZ>x%?)S*{io!ZnZ5Ao|`hz`TM z2*Cjy>I(RqG88ASK_MAIVwr0_-|a-zY#>tINni(5IF}&s`*0Mvgs3-|sHBbxx6pCm zCKcFo$p=YV2yO(tTwDUdk=xm*xB!R&^;F7%XzbkqXoDdu@nf{(2+(a_NIS4munCFW z-;P%%n)0>@FP_Sw3?U4y zjs75{Up|QLyb!YaoB@b=-Jv}-5c({P~sW_$jTCWIy}j*A%iM?dd+K0RHJPX!kU zq!Sn_2YkKN?*Wk9gl!qv!|x5@q=LJ#DYS=u;hCp%w5Bd9Q2AwoBjDzo5kSF3iX^}! z39Fu(sZ=(AW^Uk8)}T@Y!kK7h00rO`&gi6jJv-e>HMfba^82cnC}ehmj^e1{c=L2l z(rapGJkzGai3gX>UuX$vrpC?5sGwTZ6#(=FbqmQnC{&-n8M0k43&G4N@BqcUQ>##g z@6=@V1_Go71(7akvvi@H0I~ze=%l~4)7NrQuiLd25>?RKI#*fd!u@|=S-s=ff~U<( zZ)?B)v+O$$|1sZL_lr|+t#^Fs^e5)bu+G{1+;5Xk{k8G0@7?>;>t6iS>2L6J1Yri# z*lr?CF?e0(s|uBgAk=Y`vR0i@8l022PoP#u3LaNv=c@zVO$Zu*wozH8)Pum(#2RDT zG(Zr-0egXxn5ZS`7)y<6>~$qbW4CKdlN0-|jOagJ3J7>3gw;5mJc?nseFPA6Q4LSs zh^5t0d?cZi8%Xl#T3XYxEh>27DP>!ihX{88QeYrQyu<<33eSa>QHW+%i&5F_v? zb+v5ES}(9K7j_WBsZWK;cFw5r$mNq+@6eAJL?5V_53_2`Z_<;J&hc1~h{q+~sv4SG z%IB}G^)?frsv@}vDbk2&tl}#)t_)??UqkxbQq}13d1^=sT?K4IGpW-PI-b|#IvuXZ z)mdlki3JA||MuOcfBX1imYsI+`MaXX#k~tOzjEnUdbp;#rxhXuZk$t)o13R)Dk$JL zN);6Fca1=1*q-k~7v8H_F zJ0q`ah~e7paX}d!&3F%ZDZPgAsd~Zjsc~QpNf!ZQ$RehS%MigXr~8&Tby4d18AM3l zG*szT1+N#}3Y5^004<>t%54ZSvL@oYHQ`=j;(8)bp}N>pkeHYVBnZ7jBy&OHHtkTx zlRq-1Q|kK3RYL{0NYj&|et1anQH`*oXZq;pp2%CS2PbH)8|}GWm&BI8`&{^iLTQpS zVZLDsK(gP00@#@0&eQZI)g+(?nOY5;SFP|JtI~1Muhjb>a0f}JOQj>TGGxC>QUWgHLd6})p>I3<{P2h!#dYPZV5t`6!K&rr&8CLK`2~_qksU_z0D7qbF5rO zmExdn7uPrfKtp>?pp%=hfop&T8CYK^PqBk85WbHa3k~2Br(^Qm$IQsM#wJW$JT%f0 zBANxFXHT8S?Mh!VIS&b_C=X_tv~cmV(Dpn}mW6sA%?{3;zC@FJ6(||7$g3%cr?B-o zo79V7+;zCV4)6}dU6Ep++{vJx6CJ$^;sof4vEDEaKlhj}AnQVX{5cpDT?JCc zLy%|{w?)B21U5|&27n`?KmTfHCP%^m+zHl@< z>WTK@h;eA6*_OoGnfwo=iBxfSP)FCa-Z;UKGSGxXSIE03Zr_6dKGp)$kY$sj9Km?7bn&R`*HtJwIiU^bD0lC{Wg`94+a0 zK-*{tk5HnM41vfvhe#%jT)%{arhw)UXbLhf*93(H*F?tD1M?r!908$?Q+J^HST%#z zQf8d3N0K;PNkbrlc3(U>+q+Ho#}HJi>f^>{bW-EFq65~9e(+lmxbs2bdB|diRQ8yT znmtN2Co(`B0{wt0DkxAT_5}qTpb$sLjmojM`W#|eh(rZagjUe4@Tv)dB5{ub0VK3H zweXvTQajF6!mCehD6^ih49%tnxzd+hD0uk?h?6bbXuB>3rALNmib_JEr$lYXMafuc z^jp8@s4CP#J*4B8z61^gu$uR}wckbo${;;A`Q~9J!S-tB3fB7}(5>e>deCm1TOWU+ zGqI%sr(mIS8@wB6H58b%)uJ=BJnPBsj$|Mih!TYeT@&XTCU0+;HP9s%bnz0#4l6tS zrF9{_qb5e?TDgFZG-6?`${Fi|9c2^!dQ7FN?2B9ig4rfjGIxup=F44826>?d?c4(` z{4Z5XPM*wKxJ5Sm(rJp+-5m&E>kz0?)~i42qs@*w>ye489{0AOV7&_Ws-qcVrZ)Pz z5=&i4R~AJ8rMeB5se3F!;1iq0c`N7xLIM9CEHQS8ic4VjMJq(C1iDUCa7N_NYiBLW zm}=k{hIUS^;q1&79b})2slN$ys|s7Das`<@ndhW?&n@`@)dhc?U}PZfyTRgR0H=Ux zh9Vj&(gLN0NVC--(D$eh@yz@#(vAN{6L5Sl)|V?(?oBnRjq&vJKkLKutLhlX7AbCZ(Ob zy{+8r7KWD>)aF#Vr%FF}+vqK~isvqKhT%9yz$QO z{wSm3ceO2RpStbts~`S1dv?z9+!cBGj+M@uTDl?)z2T%@OGLR)0 zp9VjCjbe;8fkcDApy02G)~tn1lgbN`teb2&X|7yhB#(SNT@sGG5Gv(K^_!i(%#_7?jo`vcp_-p3wgyIKCYf2#$GD?BZkZDScc zuI5P<{)?@`kJqjf9uxNpXPHBoFTBcLVYdsp_+{9c!f)6k!cO6P!taEi3$pk%;nV2+ zLE((BL|n)=v*(4?EK5ugKg{yPo5WH4^7lP#v*2T2VU75m+9L5Gc5K}KkHjJm`s-tc z*d(SUBaQ#(Z}kj%s27bF9%Jl2ajJil-%U=LSe9~3m21z0mehRFztOT2YnT|H*y@GG z8T*;pY<`t}7k`I`%|`R~t3P1$eT5l>yVzXz411D2hu=e1 zSRVTn{;6u&Z{kUP!?0I6@%9D2VV}hF*YJH#i$BNneHC?3ehQ6jlMwy)4a71S{iC$6 z;cpWAPc6q^5suGoL>{KS)$5=fTku_n_9^XA=HemkK|j2%^6{ngWB5yXX9+*&aeCIj z&!SiAXB~g1))*~I zL47yjuUXx`)A*KW?4_p`clRU9Q;R3$Q=?BU=AUjx0b{Sekp1%a@#cl>xVc}*=AV}t z1<0}5P`sW8adWR$ds3|Rgvr)xSx!-IQISwoRFsj)ii*;frWP$K%tLg+x{{=gGwnr1 z78_#vCw-n{PEn$%s3@hVNGZ5hDzX+A6_v``Yf;ZOA;h_Q4|`g2;!FSTMCa<*tCA02 z`ga%N1wahI8W5L;z-D+ty5^GPSS8J>Raij*RNE2PFCuHe7GH=KeKM*gOQy^Q`Z3cG zNmRLbREsWCvM*GE03j~1`RJpnKmx*ZO(h1`YLBm>2-K?Bp8Dd#O*K=X{&>CSSCmD& zK6$8qud6CR10^dEP#9cSLKKKDM~`0wN@leL%m{dq$RIVjq$YWaMdQdAZ~WAQNOc$7 zfoPxg9!%~icXd|Cs%2*tH%{}oT9kVeiB2d-|F;CT(!dJ}D3XqMOO(|`aKUY%;m7tW zV{#E*a$=cCpE45fpWGWVyoii5W`Z?pM7!NaB~6^5SoCykdR+9N#cG>XKSnLeZ>&;_ zkn&$krAL05i}uD}tH1Eq(p5)#Ww^^HrUJwQel6p;HUqc_C#Do+99k|s!6LE_6q{aS zXnglLg{4_@Hp)dzlr*u8B)=z)a+4~UoJm72 zaKWlPw16~xI1{SJEK$BqxYNZ$`e9YqMRG+o?_!uDa3Ll+7jqr{c;Ql9>1X0NvkJ|o zeVn+8*QWXikYt}cW+5I4I$zm*`Rj zEw55mQlQk}*UgDVfvZ6S?B<|ERx46BZ6hwkp|vM<2)WX7py|O#xKAMG>uCRIW*GFy zKNI76x}y2#Yt2$wgi1OikPU7qH~n0JtmG*JeaOYugCx%H9*q>}E= ztj|<%7aYL!PHwZb1VVZM0)0?a3(h?$4m^=NCg5;D_BE=zo}Nqxmo!Vzku-87(Vze-bw@lg74-o`cPv8_rOY}wpq3N3G;9yg%egK6LgdrMF=!sei5Jx(EXOax_=bbnCI zqHqiTD`Sr3#P2rw=oj-8Xb$*=0;C?;1H&MKoSuMo-9R*3nV);W=$QaPv@CS|BA%s? zYZP!tj7&<#Cqe_ODp{9{ngnp}o41!kYYExX zL>pfm7I%R{7Ed_kWF|mjcAz-yW*k>It~W#dM;CQfj|V!d$?T$<++Cs20PL;7&WysE z%)}~K1_eVUi^&m+@EKgj_35xD&?$5$SK_!>Tuc0m)r|0DRP@4;jjhro zINIa~NPC#S$&7eVXX+)m!6$hh`iRNERe&=qw4?s&(#h+!q?LoIzt^}KH9k*_&0M69hF*ne~cyv+J+7cZpmIb5F zkwoO!KwRxsAjB;&jNOQxucE`~Q#o+oHbQAygY-sKPipjfnGfhpdoBSEyNz7d$tPW- z^fm%+prLIV_PT`-7u{+=c#A5l^!v++Zr`e5t#v?98zM-Ha!|^8LRq-nh~umv%4`Dt zUQRelYnn3Bj$W&wEV^ESss_F)Bc5HwQZcs^ZI5_aZ1C4@X%20vQ=KZyd~ht}m`QWo zj=@RlqJvf9D5y%k0SW#Uj9n4^Z7947Nc)C4WHxZLi7sJ{pwnef})QD$wjL~}}$tq|!AsKbp+{VHO3Q)a_ zj4%pkm}nD#F7i4LLwYd1>t4e1S}4xBEE8*w~O*y9G%ARLL62TY`DrrfjLkr%eSedht37!I_H=! zY3P!}dZLX8W6;KlAhn3wy(ZGor7*j7Nr);Si9|u-Ldp`2l4vD1rU^j(Pan+E;af1PhE>`T>V}${XQ~_ zR*&vCtf%Xt?%%7I)9YQNm2+sJ?)N)ai|h63e!iS+#XG%!dOp2gJ=FWD`yuQ4$g8z_ z^?Z6gx?i#0Uws^Ux?Z22u7~hC9xPcN_6ua8%6e_~uOuh*y7 zua8gfzn)L8SMRSruX?&3>h0;}^iZ!~FR!B49_sb#aeX|xW<`Hb_0yN` z@1U=)@W5q58;Jjc^w&k00H~c`UQ|h}E6`|(mQL4;k&O6eFwI#gPal0Q=sDzv(NDbx z)(<(7P@RfebCJ*Ia+}kB}q<|93nrL=b;RDSl;C>$ST-D)!{Q4qjB-?IfUQw zgzV>%uLlE{O?c04T65%sN1i?s`LOHmIx=$PNHR-4g1eED^Nt|EOO^}|i3?YM@TK82 zolcR_99Q^*TfnlB3Fc;Zz5ZB17e9eE;$`>@!dxz6NqSM?qNT-|JfRwwV}lx?^{$TC zWW&T2cO2;wPvRAcwTIIe&%9B+56K;(5E4@OBkr*#V?J)FfEqjwPj;SXd5$M9mwP!h zJ8=IIZm??3%jFi>b<1<=@^Tkeffv13qsk(pAK*PQ*1Uhs;v%pdcidS}a4!)bpau)B zBcUJ!CaV-EVIUZB*145yl{#NKjOgl|-k|$lgaudMF|8mWK}eB=1i>H}Z!oM%Sehaz`6wXCl3*|=it~j6DlW;$k|@p- zL~*4ciION948j7@AWWmU5g7!VIL9PVZqzLcW<#R5Kv;#c7L-g!^{CZgGzgMeoW^TK z9|TmwD3@S~N}*(h`I^Nid;P z;#Hz#uu9YMXcCeI;BG0^Fkf6C7K?&e7Q|@@!ZOs?EM6<@GDu>AU^4s+9YAl*RHG=H z5=Fs&O}1oDlnjd!ETSTbCI?=ikkKKFVuwMPE|{q?gLtHv3Ey1E3?CM%6=w3_-c}(| z5i3N7QA)y9qEUEUoHuj2ut>gYnjmBt>=+rkD5MGNG4rBm!ML)7CRrBE3#9!9mM`VV zLXt4YC?uoi8N61bOE`!!t0gAAgZ3KDtU7!eC`JEZW1*!ri&@n_cg}V{PNI-vfaN*WC4r>%Seo|9<=nKRxreHNP=z zNnt|K$1@W?@w(w{$L!>+k z;=hAcy=HR)^9tPUShcO}#;TIC8_O!UMWOBZ9%O>Fp-X_O;F6oJHO$>KR2QG~7K5S> z#i&a$rT7ZIzG2>K!?Ah)^^La%-l2r=9C_l^x$iVRGxXMR9^W$5`}NW1caQz$tsx$_ z{AlOeHy^$A>oZ12c)W1*fqltSc>F8he(Kpry6*m@ zb99u)zx(j-uiJg?f#2RcD)Bqs8n?9+KUy~MrN>4sJbv42FW$LlX5oX+kJ@-V|JJuQ z0+>XTDH&?t}$m1XR@`|Nz)jPgi@pcK1uUd2DKVCn6v~}m()&bR@on~R+kgP_>0-u?(NrifA4Sa^z(Sv z1NROd`Q@5V*v1BU+_dV3rTx|&Bbj5zdHe?-4SGJ2miBbn*bt8=eTO~t$FF?w?b@*s z9{=6E86S1Yzv}&$u~R%g&{pA380c;P#Ml`g&v0M$%iHIE_4_Z3jq*6(bgcNmYhya) z2-(HwkDoj_rc<^xUElg$%d-ZiN-A0t_MdH8xn}PIrrx;r!ff-m?|tp@J_k$VRW|W}%)fix-{#-h|3{|swYUHJ>ZdY1n?5>S zaPjxQ{czJf*YAa&E*88ze$(zZpZLV*8b7;E2=Vy)ciw;a-j;+v?hx+d@!csu-}9M~ z%EP;bRv!QNZ#{X(M?Y}K2R=;HAmpYHfv;E5xDnIj(N@z=ioo62pk41G0Q z?B(%auD$k;bN@8jRxbAQ_~%=8{NTyglz}>NfX83ow(#CRy?0K>Uhz1O7uc_PeCR6& zfBB?1#N&^>@Si)jeg7w&FNz~PURd&lLqECi$+rf@Q#@`hIrN3{@?Bs1op^@F=N%}1 z@y(As^RUS<%HtWWw>@Qj^O`{ggaaik$$s=VzdB@na+Sft{RT7SLWU{l|p zSyl5BCB3W3kjmrlKlX)t-gk3J_+djDk1zXqm!WuM;VYjvWbpVSVd%G}9{!vXwL9#00?pJcwlALi8b%E;9|L1R7c{`}A=3`l*1f^KY{Rl;H*9uph5*)X zciZh7?6ef|L^k7a6)u=VU20a{st8R$R7`S$dYBDxe>>GOlh}D5RLrV70hcYQd_FRh zv;Z$Pl?1nAAZ-K53F=_QK_Vtq`#?r+F}e7!eVsFxW`^Qc$X0_(#7U*eBUBgquB{$7 zQb06Qm8`(Z$6dD`zZ)DeZ~-$3>kjy;<6aVSm7VxrPgkeo4g@+IxrJYOKk;%YrJAlb z(Hx!egSyR)+Dt-9Ld+AHpy2n$RIW)g&QO2zK-J$6dk5~0%T7qlu4atzd#V^7GY1A# zzwMS^@}r6ARjqDJuBcF77v9&>rN+%ot5+{vt^SY=Jg&)zQ)l*IsYxs#u3}Zw=|M{& zD`Ln{GVmu67M=IPT0pr!ga6IUOuS#YKRsO)Y-S`fvO&hj-eR}gv+X(d<@Q|r3VWVC z-|n!l%(latIwyO1c5e2H?7Zy!Y)AIW9D7c7PEOA9oZOrhIe9txIgXr_%k9gvm**^B zzC3sNisgCB^Orl8ugtaQX6NSQF3-))U6GrI-`{rRu3TYXk-Z{k#qt%oD^{$?TamxQ zv0`POJuf>iCvSOPZr+N#yuAE8M;?BXJ3l)=Cx1D9w0lKtd z9V;Apj(ms1v2rE4xDw5;MAa)%Y$YXK;)feG29`lfzso~5m~6B&Vv=M9^ElHuE7c~<7UzmrO;0t>m#+a=ewVaU ze9rK^*ef0rzb_uQ{4n81;*Z6j3d6=X#9vD%#osHZrL*GSrMCpj;#EbPw>X6Y^FU-TJ4&?@P9ny!=~ld*JCWJ@@j8;h7)&*N=Qknw~Uc=928( zf(;wXZrZ%X<^E9nhaY+I8{h0d^4%Zpc=4-Or<&!&X|}m}D+@Zmc;Y7s`THL2G$*bq zs`EVZ(HZ`#SAPHIduz^&j%}-a;>pac#p&BWdGIrzJM@LlXJ6{=H%+t5o?ozL!!1vJ zVc>fQ%~#F4_WGhVzdrTm*td>I%5~Q-N?-0MD7~q?V%zpxZ-4KO_f^-r>vjfP?tkDz zhn{|}yKnGI&-wg6|Ha4NckR7KgS5<0XArV7!w=3kWGAOeX$cFAHyBH#q^04fO=(h^ zlrHB^+ib|S?9ES@lPIUG+OX14D<|0J7#A9@HVVZK>1Jb=lxR*c7b}Y;OG2KZz&Ot= zS`OBv-H)qO;rbUMNvKSuRYBl6%$V=zUTa;>>o3I&8l}x|NoMngykDV)&`|UAtjgqG|f90#l;NmYN~W4gXuAYn!DsA#vk| zt4rl=)62|>;TNw>N;GXuoNu_XEZ>lXR#%!6_vT$?US*iST}aNE{?L2ii77VWLH%r_(&9p+S1f;s&6Rukk>Mx)6jnoY7fVMb!=w5u%hrdyLN z$&%GDbLOmsIl^2iMYzf^&wRCzDqb^3G2CESHZ4=IOW9(M@CE6M;?vSI^54a`jBktY z7{(HwZQ1kihd*n-?e>S;_oeX)q_{>Y;r{o?a4eeGLEzWtqF zjQnbhN&Eoh7pz)ccGI1$AHmDRFMaLEcV0U-@+%fO9;@hlywBxsee}sse*3j!(`PIx zShb<-wmWvb4;T7=_|Y$-$hW@z`pB|p#jn2djU%uAc*dN$ckKAfU*8!EH{Sh= z*ORXC`BUdt-TyCNdT!I#dgshtaLvXWH*cXs@-Gj3I;~n-#u6ga)e;j{pZ1a})-eH!Lt&6jM|EABMzk1EO4f`J5R^NQ| z+k@Xf@snS_GscvvYae`Fday*kS~AVp+m#f4#(0gqH`Q>JEJ#^Wu4Fa{W|MhFVny;y z^LDdAN=-~K$Of|kVpW4hG8(3tgrwQV&E~7kw?SMx*HR&^H$Z?X%`hcf3Z(hhS1FCs zUDt5N>o5PSjEkl}Tnq~-h;Z;_4 z_>5d=NH!Efu6cv;!Lb=Ba@LGxhK0!slf&)OgHOzvHoI-VF$*#|F)1Y>{Pk-?mhg|} z?X?=iZzTNjQ-=J6y*q5-9y$E;*{ckRCWpLHwwOZG78u?uy(c03P)cgzoP=^I{2|jb zhb(iY?9WJhf4azQF&e{9S@-_gEGSD&c>7@~{B^_ChGeK-CzI4Zze@N`e#1%JAlQT4 zIKm>;surRmLI9f03K6m{58-?mH*$hC<^_n!+Gg0^kd+QyT0{C6L`yO0xqSBc3rN)mJjx@r{wDl1G3f|-W+)9+yJ)qjRu5Q}2(hC&84;yQ z7fz8~==iMES@25_NeX+^Sk3O7b&$=Rt6XDIs;_x7L%Ct8{ooD$r>Zl=&f$y&Zw)VF z?utXSObIzOn7}UCzoa z{x>Q=+3VcSjyZ32fB!SLvY#%zjSYY1_AmW%;d@Vwym80iN#_oQ{eH)3;ek7KyN&sL z0vWs+)z8FGr$Y45tmPwmgBZ|Md@&@JjxWVC@Rjhzhl;3=#naeH7z4Dx_@Dua-AfFt z_sA@4VXP2g3G%uS9)PjM0fY{Pv5|XZ zHnb4!-z&3&*Wo=v8dNuhOTnYwCo{Pk?LQzhdktgz5uO19+l#OgyGRZr+==%HcYKDi zzK77xbBqnP%FO#b?juHMhlnEW{W9D4B5biKd%A5A8B5oGvgs!ajr*2wRc_7DCtooy{SHC#(YNN0>2FV1o!-XAA5MLi>Dy z?fWq5zgA$jkD$NTqa4E2B?22jSP9WY+9T*kmcZ^q7|s`1={}jAT8Z%>JWwjI{g2Ab zwF&+GsLTc`1hxa=$r^!GeoSU*F4T|k0KzVW`|B}Ygcgs$_n@B$q03?WDSn^8 zdJ*P^kssmEUV%9tm)W6*@E&1CE80PL3Slq8_V)|yD8i$VVVpsD=z{`F?ZAA0NMK=v z8J|IZgz2Bf{3Fc$oWMp9?mL8jd;;tCC5#{8(HGGVgc)DMydxa?4(9g>jC%m$KzMjMj`F9lZ)S-s7h!9P$Vw3&zDi^+ghSVgER0Y}7ug|% zol8YFitsc-`O`8RzCmOe2ve7dtN~$qrpQ_mD%m3ILD-QavSEa05so74%@tYYL7a~j zB5Oi;JWpi%5VqxutQX-~hsaJKOj#u|+h@@38j&49cxs)<(msoGzCmQw2<Vm+$6Goga@~vJ$k=QWTz3b+o2KsoXjfThkhd5Q6;isgbmf`2g2T3k$Dl?Tq0{h z=tbCy@D##*2us{(AK?Lnod{d%L^eoay~xfYEWS%*$|3X{VF|*%Mv>izu&o*85vJ`C z*-RGHg|PqYn0JItuVB6qruLzJghvs!BJB8v$T|?V zeG~ovf3%$sT)*eM|L^*Is$1V|u~;k?VzF2(ma`BiN3#&cIu>HFSS%Kc#bU8oEEZxR z7Ry;ImLq4OW3e0|7K`PqLo8<<%aJ1%e$Ur^UHARD@6YG{R2#qR@wj_^uKT{O*Y&=x z>wW#X-|y@53GDbLd|=VH=m)U$X7uc{wAT`Hg2n#;a@b?U(FpVHpo;yV2UcKCgPP%hZENjreUzeIn)`Yp-@d$-ZkY1-#E)DJBFJ^J%G+V%g)4{rQ{ z`(VkRa-9Wm1>6K{{*3;9p7!`F{Q>Udy`2F&@)OPuIO!&wqU$O5J_)A`tSwA9t>6;a zt9Vhu838Bnn{Z~quKOjNHL&6S31=59ElD_4Go*U}`M|{oCY)h#yOi``>&XeH?F-2F zu!J)PZazHWEP%z2OgNQar2W7KaPm?7$2<&}Y z!WsM$?fLYCGY5`710KbnlW-RQ6}^8xJYS|iUzl(vgy$rj4Y2iGPBP4)pY6y8Zh`e+ z^Gn%}fP3d9oNaLMRowpy@}8e?I>0%w8+5uz2X=yk;08DXPP`g9!2Z`HoGq~O0@@!e z?@l=7|Hk|XR)M>(g%4bO9sLK+yguQKf=#`&AGq_zgtHI!zKQ<65xK!iudU ztzhw6$PZS&E#Zvo{s8p```o!2X1|s z{sPCw&=;`jBk+JD^T-YMe>LH(f^}a@ID6neSo#h0 zbRprigLO9{FX(&=Ud5Mamz&Vv?k>u%EN1Y7TubOykE zZ~|O8mUQ;O(W0bNvW)!qO){6$PRAvk4sh=Hq%#gq7bl$sJ-HJ~8R^ zf=l24SYJwda1opbS00pfHo>+BbAsi2)Dx@#_f8@|*!qy9GX!p*oODLP(ek7-1J*q( z=`4er;1)Rf@T61xed_&)q*Du4RV1BOaQ#t9rys1SOgdv=(W8^j960tE$_1NFO**?^ zdsWh@USXYmY|^O%$G~=Q<}~C5i=IGwu`RrL-H^_%h@IH^5$Sv4eVnr7x$w!7XqeTz*B;SpzH1Lq2c^EWU+x zqBH4~f#t7EI+frUSOd1dD(N(X&F3ebZm_Y7c2oS-NoNw=2N%H73zE(TSoPYZvkh*8 zdwPB$ysOC73oqFF#-!5#*1sv~bb+H6BQH4gX86E{{-iSt&b}31u=EPzKLW2zI*Z`s zhiDIQ^TYHH*!Yp8vky*xlzRM_a<8Ucz^aeYU*O7DX;(1$E%XX3`5xtg9Y3NTVEsD% z0G9rO{s)`!=`VtPoWj}yClYy1(XHgq%X7-XhWq9@b;1+!oDQ(0B+nTDr@;|$^8tCz z1i1I`JZAyyZ_RU7z&cKwwXV@$*W@||Ew_>rZKql{}{wZ1`cG(+sYI9pLg>p3?_*+=g6W>wo1r zQ{ckS^PG8b>=${?GC2RMJZA&kyd%$XZ$lq{o##}8dw2y@|`JghIjKt za5I_jtbvVgzOx6m7Ut8I=+S-ioi?!IxO}Gz+yMK)iQ;@`3~ac6zB2-Y57hSxc3a?18bg%Twwn{|0(491@SMyze&C?<~vPb?G5Ay*T6oo zX+EF1n|Av;^#`XGXh*Q^-}9YSu=X49fFn22o?!bospl`zkDK$IS}^(Te5VoI_zvZQ zQ!DU-OFzhWhQQ7r<~yTc+hYryX>j~;16l^_}vKZexXD~N1e!HGZ9_a7(p=2j& z%5hp`CmI2_v4}Quzx_DJ830E?rC$K&h|7+&K|0xys&kzd*^zh^YLOkOJmIuVU`JwW z)#6}BS`%VNVybSD9ccn=csXnwj~x-hyA1LpeF@o@P|23@^GHv;2s={`*Z?YB4>(19 z7aSp->`z-l>`#paPK)eMGob8GG;zzojN|N(ei>AL7kgB{>m+2Cn!S(Hvi3#N6K}i$ z9&i9ux@oZdKFEXJ$}MzSWVh-NVz*iVJ-gL0r$u(F5l|nLBQATFdq1Z|_O3=y_O3Bd_O2~Z_OAN-QxEK2V_;h`Jd`K9*ADUK z`yns!DNu6Gfg8k=*uxZ;U99m0r=|M@-Je(YmIpzLECpzLGS52W84xev-dwgJjMR(+z=vicHJZt|trXHIky`=BSs%cy6m zlPK;m@#>c&&x6tDSJ3XD>}qui~Z3mQ{t@fc#i|lLzpzLfb;P{(zoSKuJ#5&jo7GrOdyt2Rb zKAe65=Rn!t_CVR+>L1~>^kaWhI@#eyh?l+zpJl6KQ1%RWyc#KE<4_e?t{uFdtUJ)(Ytq=xa@k(#AVl;0%h0R0Vjsc z^C_^DqvuucCLK5fif0xaAifOFfVkjOx zPoRB%O+TFOw8*a61+H&X@5fWm|6zQB4ZlGjz&&sSoWG0bPjC|bJLEr|b_W-p$T<67 z_@BhQgB`RT+yW(MG_SRKlQY#w$0(O&5_ScFt8J{k8Cs6j+WF7Mo_Sas} zDZ&0(=Op?bVDim^-A|#PABf!!+y$4x%@eUtf!(Dho$R(tPlNA4xlVOG^XG$+=c&xk zC*?Zr;7plGw*U?k?>*V(dswcsPF!~05%Ntul6+5d65Ef$t_I5PyT<+EM`Is;I^*^+ z*u%haQ1y~Mxc(n$A8-JaJ$MC_J-EDqc?p~#{R+4N%0Aq~ec6X+K-q_#XVMQ~GbsD; z3^?#a?4!?c60#E){}c25ldww@m%X@$xa`F%;37D}{mEJ*kL<_A&!YdZA9sPWA1{Hj zA6GWAzCRN=LD`R2K-rJW&!XMVqJ7Vzyk}$o1e=?%7e1T$`8o9CvuPLX%4KJx|If$n z$$i!WSx9a!GRJopyu&BSGAul;AnYd`t{j=jyK-vp;$ zjQp3P_uwK}bT0A?U{3}&z!9+K?ese+`+RLX?fFiVUv~Qam#_~4)&BiMl=~9K3)l>{ zy&L@kH^3RN{5>X}?D~B#V?12J`0Aj%VROId{j?wP4R9Om{D8T?1eU#=dG#t2p94FH z?~Kwep!@^-FLzp|KV+UeWAxuE(7%r$&w1!ENJzRLHP36n4(`jJFnu2Tt&f}gO<--O zlPLLwiObJW{7S~vrxuf7-7kt;7co|srYUVK;-v!%=%WttvKKU&wx|xr^01sIFMdmG@%YU&- zT>guS*D{W0DgQOhgP_XkGnm`}XJ3nceTja49eM1rB)Ttu$;g|S zf4;|f1d}V~zWgRLeXK7(WISEWc)i8kF9v&wd;S!jJ3qqTMqGZC);H5{KQ`&g!LmNu z^H#<;D1XZW&*g6^dJF6JPffZ3aNsTIJy_pQJKhE#@#=Nj6&wI1r~ER*JeOZ)8{4(2M`v&@O3GEEly^Z$#FOy$>nsMSQp!npk*(WZ4O~a+=*Dv7#H-Clx zkY9eAP2yeKCcXSObpy2b|IoiYmmg=4_`ojZ5|=+`mAL#l6>p~=f1sVfvOm(UZ)ZLP zi{8OJ0XBg0^UPd^JfJhkzUa@)d!YP0Q($o}{tVJpCa~LsBj7$L|Bw4l+C5(Ib$F z_xw%5O6>NadQI>D@;kM?kNNvC$O+2t@GK>44RK>43auVTJ_2K@ud|Fi_k|5Q0j zzEi9NQQoWY2Z8ceErRk_m3;&~>OxOI`KzWu`K$Ip`Ky{fO8s7g9|iRM zRiOM;O;_{23W{HTt4ZSWTkV7LTQ!Z--mk}R1y=T&bn;*A5?{0N9dHymtKLZef6Pe? zgW?+l`-#iXRr?9r5&zZ@C_mR4C_h)_1n&*t065!++@zPkYlV2*CHQ%W5A@>?`#9t0 zZM5Gf*5WgKaD;Qa{p7je}EI<^gC&Hu;X3uUBmdj zoabN@I0DL#)aSZE-as+InoU(tRTyPtlzMlQiH1+%<6@()gf@(=EV@(;FumG>WT8kB#q=xgZ9FS!rOKR6A_Ke!Ld zKUnv5_E~o@9zppBS3&s)s}@-Qeq-d7pKy@4{DiAu>F=rUSI{G{?BB_UzpxjSzi=Mx z+&B5;H!S)F^B;c0R&eGo=Dz%goi{P={z|`twGRHBZ}6TE?h%(Cv3U`>lS#+DiT5CI z{+pCv04}0O;0`GNqWdlOS70M3|Kb=Z|Kb)X|6=3K=r8`oDNz1JcZvGn7k?Az;$KvM z%g?w(Tz*FP+q5%&M%|acv5|P?{Y_kc#}VQKCnTNnZ!$lCZJ_*+YuulGfR`>=iXT$> z<(HiK57tvK`5p8ae`G5tzvLt+zhrWmc?*AJD=5F@Bq+aR^1IaIA^69@Ik4tG(C3HZ zKLtHMCHLj0TnFW+toR=D57-ULPdN+9Pq`1uPucK&=4nuJ$zM54T>i>+Q2xrs75MO1 z>bd-uR1d1cedqP)0oV=d`5?GJe9gvpz#ZbNZ$OWK#{7Q~<$z1z z94LSDGJG9xH2LLsF1t@oi~P==;4-*Jx{`~L_dl6ez*+A1_j&1)^9FNQz&@VKKfTNI zj<+yx5MKnH+tJfYXeV$El$`QkHx<%;pyDgw2+!ru9_N1f+syq+uzZ901ndBpE~Q<; z(E-{M?07rR|BLuLIA;NNT!!9&{e#F2HoepEY`ly1`Z>>sym(Un^8I7z$Cb=4MfCes zvCU^tw0j|tZp75)T>*I4;isL`k3X7pOW+jop+B4W7`R1z6iQEw8*bN3Cgd(56Z9KdVj`uL7vk^y5jrbw^#aOc~0pT;~jthEYExHpNG$s=lK2A ze)9j9oIri>`*(u!|IdQm`2Xifr*iSV4zLfbDCIf03GRVY55j-{-^^bR z<~dk)63@X6a1R_Uu<|9`NH zJuc6w2NzE>a&J7I^8mzKpU$}haO9sz_Z#-FO`IP9H=b+Kbv`f886&Rq3-jPQTaOp9hDCyYE9@u;~4qZ`ffxUuB-_JVeO@(65is z?%ePE80`-BOz`}-y#HL2=k)Tt`&yn8U;7uHgY93TU;dYNy3ylF))~wl00+2V!MTb# zQ0FUF$*=Pj6%V8z7C2V{_J0F8e~12pU104^RvvJh=gCFRNpN51FRD(=Y0>$MUU24H zHXY|M)_AV-7>nFr{5I{k%Y4YWjGEG%7M;r&26Zl@=s`IxI+xK6>RiSwSihR*G?IQB z90BWYg%4cfyoU5j=QffLW_;Y1=QM$xKg)B;T(uH|aXTDdL;p7TEM(=DyB( zERwJO=bYpCKh{Aoc@p#eFVJ(ab`w43{th??>iowf_jUeb8`SxanzEdhn*WCH4~&C5 zJ>KNnHtj}y=Pu+1OMZ*~fUUbc-=jVMfc}8%-~g!eBmMA<-;KTyUj&^$Qg3jT`@Mhm zcys%~YVI$A9bnC0OnGDA81XF=&u#mwci)+C@}1^CF%CJ8G6d>e%HTtC?EC-uIr{F1 z+EM3J*14~9Disf9UMukKCtJZe^6A`4FZVa^!+8thI=|9>a!$+Wv3#fUZq}b7!;=Kt zh_8WTpw72Ul1}Ga)%2?F!!$2& zeq|qh2RpzyP;%=WOefDLODT{0IuA2SJoym#h;M<#e+D0#@AQD%598bmsPi(bq#Jpp z;h6)A|H5<5&(u7e{;Y-v)cKiBQ0HfA9>MrHgL5UM8wORr&eKeDU*~C(6|7gBr|ILl z&eb$2&LIY+)A^cV;&o5qToBJopK9)Rao$Ej=WdpHu5&k4kEHx(k{;B#n{BZ1S^3Te z`Su!V@4upFXVDJe6iCRO1IH9UoAW#%n-iy+?^zB$hkApZ&o$}Bz!}AxO%!1t$hDYam@a?yBeV+44@Xudw z?(3Y-7WZ||r|vO1Ei2&2qjM5E?=whT=Y5vJ9Z>bC{#W|pROI_I?F{aJ#oQnLH~NEk z@r|56s>(^|e9(9mt0o=ih5A9A7g`1D7I{uS zof|59JpFnz&q19VS_OOmgZ_JLPD1C0DxSbP_+82)J_Rm=Yhc~uauQ|VV_bk8;50Y` zu7XqFhxat@uNb*H!A|1iHm>tX{qX8M(jutyNXgS_*IT^w$&yw2_w<~^%8yO{hFhsW z@i}l699T2=N5EuFPGSeF1M7c^-hlg{+N1n7^*Ge%_qKG`I-1CkmY8 znK_BcyaJ~kTt3G6EpWJ)`*rl!37o?Ms~=F{OoBBh6*#-#NLhi?@RXdy;>iWhEV%pV z0{pr;iH64%I5kftKdAPa13QUtpK9Vxm50eLgSm6y9M5%bY_T5w=iFG&)96>QWbSbhQWfGgk%So&I?Ka24SHiN@G z)EC@)1M-4JZ!+c8fYpu2-&f%Dfh(854>t94|E!!u^QDvv4!s?Dz=}be53GMS?Evt!#f)}!A5ZRa^wP+-%Yt-$$L!x3b4G1@%Ua7p9lMhcfJoEu=&aYXAdkM zG0%suLZ0WKUmrE`rjOBX#NE$OU$FbL=6=<69wx^=2mf>FhtC%{9bokr%=2b&g!tH( zI41}W&LQvfXy>n(=WAds@wR_M55P5WOz|5@uQ<4;_&oa3OdMpO-(nT)oaN4Aoh7b<33pb zbMAvfchFzu>^s2fM`2&*JY)~3^N`b3*keFKvgyAWKd0i)0ms25aHE?2!=1?W1lFx> zlWrKSc?$btaQ7M5W&WqYS$`q>-rqA$&taY0V?4i@{oWr7oI0=<>;>mQjZ>YwY_g!8%c$3%3!Lr_>h%}e^A+s(z)?`=G)K?J zf5SP=T~OyV2fMJZ6>&cERjjL=*Hry`uj;&wMre{rhw7@->{d0S8{od&LR* zjtci5z}eQY$&YkMM&Yw2$zO3`7lVI|n&HYNS zR_|wEAMaI@FE;l#!9CtD8rw}==T)oNXRd)uf25sW%6;~eI={Ni{!-^xOV}qabeQx> z&apb|AQ*Q&VAw^p%#+2}NJopbFaUUI${&mA?GTm#omj#J7tb2>k)OXxI79H;fT^2VZv6n{MDW4TvxnzOIy7hGR;r^3~oWPc$6e$fn?)nS~=#y~5m;G(z!l?AJ-!c53taP6+rW za`2>^+|zI^H5pzf=@sX==Lfti zxd+R=fpU^2UT*1V5pJHYX2b73DT99@(Q!aW_=-?-3ayjqh?ZhBL5$L0xs%;@nR4p< z_CM#-zP@$DtM3Fony$SE&v#vyn=({4FC1aMOTOFWxu^48oy~Vo=esVOZ_t-9eNH*z zd{_M=Be&Yu3n89E=DRm`+WD@TjOARTwr_&D>W*vtzKN_Cg@?>ll0)O6m&s@9*ij ztFrn2)^S($us?WPfqOCT{@jr^?%IB6+8`Krw+3_Ky&iXdy9VR#x`6X<8h0x{HnIle z?h^s;-(}qO{M7IV<1WR&@b?^d#l@!o%FTDk9QVp8e>@y*+zqiCIKsGFdWb0}7ctJ39n6M8X{59B4>x%<_j@6C81mLPr^}mIr8DcBQv%1nA3XCVY3~Q6 zw%v!VDDGv!oEjasfnfgIf$0OylO2|0-RC$z3y0d&e@kPZ8;&Te>O0Hj>q0%B72MQ3 zAC)OWXXM!>ec8#Ty}G^J3eo!>UlV^9DW}Y?Q~rAArsmUV`*(%g zIMO~IUn_r0*uOVhzFuGM3j?_`+e7bYkp_uIMJzcvmg>yd`Do0JQ}<~BZ&c1iIb${Z zK+rgthi|XKv~Q&^=d-f#<&KBxJb82+ZsU$eC7bf8ejI0V8Xjf% zf_^w9i#$i@hjIA&EZ>|8A-I2-^w*WAl~*UQ5N1wc!w$tZ=a!c-183L zrESCV#n77mhj;MNhPU2thZ6(djPZGO!8m4?o$&a)ZQZu4SAX18!&QcF+A|qzya(?q z2irLkZho9eL%fslZk%p-H|RSB_q0IHSUtG%=^mj!%>^!Fgu&NOVP-lOo=HyPe;f1XT@`z-y^6W1-? zxZH%V{kev3#@`3uaa1|?n1?gv^y<&=Hq5pg-X&izuMXzBh3s~)#FW?PF@`X&{_s}3 z-0cF_vit(@_8D(!Vkx%PTY z*?V41_|tG)?3guk_e|#8oqh)jwSOfB0hi zz1Z|Jev(YT()=5Yj&;~Ha0h#}!eAd0eeWE<#N?@s$)oqq&})U4LiHHr^7HAl7eZxu z@2iK{OEumULjImNKF_|-HFCe4tI5_UcvrtRu(1dAGG#@!d6CVs^tqd>?jlpqK0YhB zAI`$n8Qw8QsJwjMewI(2mM`!{otK3#%@-wk_PNU5WaUAJ72I>OlssjurEPCb>(2w_+0T$ide#Zg z`r8c8cBVWd@%udfrwGeK+BVPIhw^nXr*FXU$#JJ!sL+w`|&X>zPTp!9SdL(QJ2?wS+WIE_n&hgq|?c&DwE4v*eHLql{k7lfVBxrhS6^XC5toE9Gh2tlRwRO9i(e$e*cK9K?tyR)!)GMF4u3u=l&*Jp6N*t5(rVn5C9&(J=f5n-KadliV7Tjn(|;88nw!qfOZQ;(pX z&k1<4zXxivihm7^Jr5_DHJ{C1Y07F&y=%pmQJ82=Mhh@vYwqsgnzwUmm(>mC;QFA= z`2V8Jq>)C`7}Gqk{XtXythEi1?+}~NXkIp3%NBgiAG3U^`m7xCPCiS;NE81~t~OE# z&G~0NBkf(O^BPlTpm(=^)Ysqmc_()+y0Flc2kps9KV$pIh@$na*>78>?`FHac~8}^ zYTLHYnLJaBTQ4*mw-USOO!Q$A;!&#lUUG4aa<%`v%bHeY8dUoggVdCxIJ-1u6xwoKXd ztEu$CoShqN6HWSM(l>qE$km%lpW2gXYP6judG<*^V$=I;1N8{)t(dRu-b(eZqEEMM z`iT_(f%R5@YEL$}Y~-3v=|vz{A^*o0$#@%;f(qGj89^J4$v5N+4VUB)$B?xxRi@+! z+C_V482pvYOQC$*X!T za)aQ0IN%*Q|<~9w|4M!3B zw(w=cyXx~k>nOarGcmM72MlC=<7|Ea*k#jid&Bw;n%pD_>QUD2A{offURCF7~PhbEb14HpMmchuCf!% zea(A`gU|48-w!>~4_}ARS!>c-ioI`m>ws0Rstr?Ktu2o7BJ)*YYihy@zw*b-Cr|$e zJD;?{RnFyF?sk7ZxnR`qi!5WfKYa{ir;85ixPDMwB+t;MnKK6X?1h-~M=}Mc-tSB@ z<(JIuzcOtxYi?`p?8vmY4X_wS?QP0G#_=V{Q*Y-Pdq^-ql5JGzp^_@a!Px<>ZtS&Wk zznrV|b|YJt<`RN?_6K}jS~*}I9A^RNd3bDKKq7xHSJi(R&JLgRWdSF%S!liUzc1lA z+Y;i8o$aIt%Um_TFuWUlR&Xy2c%x%0&Hfvw+g0RO`*hg8ReN|LG)~#`nmq*%lwR}F z`hOc4)VE#;rDtZ1r0*j^*_|eRWvb5tS@}|akVE`aTqWB_<5!zVw`*;BwYwM4iS)6h zeP+s(ZIXHQN2Xbuf5~PK&r_OH{R(;YCwyJ6)lcb?g8REGec1xr zZ=q};;RC0nQ~U384ch))=d*%)bC5sKLt}#u?VvqBk-wcb`y27e23>6XzuRMj@7gSU ztOqHEOdne-`Sre2W%;IkzViaUOg%x5Q)>)DvaE&UqL7@s@D5nsL7(@;fHzuiTK*t8 z)ejBKV|$j*AII(;@ALa1$`^CCZIJvMzcD&C>&ufZe`7=*YM7EvS=DZ<@HO0J_~v}R zn*(_c&{_6m(M^)+T+3U-{`mKG=y?NO0znz@U zEPIgDmn`$1eSF^8#f6E|q>AeFC+5_5E%Tq9f3^mX%IoCpUh}@GUr^o^|9{F;|M1fw z{Knp%cP40sxPtqv82m{W9rSU6cNAwY!P0jIr8yU&E*O7%#-U`xWNyM<=F2b%fo7{piQu z8Y8}L_)6|)_?iyn+ll6n;Zys_*HnCe!&jQ(OSPwiCffFtJ}ZRYg?{}QyASN-emPgW z?fbx%a{RmfIU{~d$rpVOZ7I@|3ac}Vd{%I88akvL=Ipd`(r<{TfoqFvyU4W3qCeMO z8t{)~?mZLwHY4}_Ij-TDc+>{Vwy!iU6x?$Iox+2)Un(_ z?Al=!^YW-ax|P@Jg18mjvjeW^-X&r)Gkc$D())9OKWDoqX33u$A&A}k?87%><!_(YT6U)S9aXUTuac?Pp-&K@baX9j#( z=HZp}=@9bhgee~Jufsq6AS18Fwt{>5Vf;s%f5l(NJ4xe%4gZ+O4*y+)@%n?UA~o+E ze*Q%KI)9Kn$?(tm{MTpUkE$mxRiV8_e{G+Mz zGi?)I`O(d|+C?F>zW?lGyS_JYznp9Kv4(rjpMx)bhd+nK)G_)+2GvnIul-rYX@+Ot z=V=Le;=hZZD3Y$LztOJDAEAlsYqh89RC{_xd&KQ`D=8qd->o69^kullrQ;wPGNyHoiy`;oak+>7Cu+FI{FHI`>H z#goxj_vHIWlCK9|_Y5Q7Mv6B&52OyJXycMg)YN?71e-8pWg3)bCIGp5M6_ zo&l>LjaF{=qJT3?|6l};`<8j3JUOX(+kV~C8*A&7)AbKV_TYW(^q`!H?0z?Sd3c6+vHyHw!=FN?~3K^^4sOQfOqGRGamAsktQs(zlz1L2i26R z{%?M&sf(`*jMc!8LaQ4u3h5N8xU?pg{o~Q{Xddb3O4{_5eD*@xJi=7q&m!Jql{3q= z`wUY~k9U_p2iUeW#%pXj;d#TiCFxZsYzr%)jeBhRp%i~Gj&)Sk2xaQsK>8({zCI>> zS1P&Ju1XRbSMPn09akFrvx0kTzpq1q4BoqGSYl^vvSPS1bM(eqD` zzsu$y;Io2zVV3-Te#Ol1hDiPmuBr6$&&-lPM*eczav?o`!O`+-4(=fTn$5pXT*1BN z62JX2*NXHr(WDp86g=Hl{~bOnxHkkm8FNvg_XUk$9`)umwZ{&5E9_jnArgXnRf;ES z$6*=wzvKZ!u}L1W2}%Do9@iR-j?~*U;jyXT#_?U8N-tZ%+A~f1_Eh@%K>r&k*4lz4 zb01gopG^AEQ2KONtoMD4G_#({o8xM;`KMC(dxG*tFnWh<&s(HfvFV#r={x-Ns9A?i z9yflT^T4!qslVRHE;$t3v)|(P{{j11_A4>|KDAdb*M{XB^7rK@1)PVSjoQ>6D%<-` z6Wps@jduQ6=d*(QyEpqXXXu*QmyE_OZ?QU~I!I^ctq%M>_xF;ulS?`?^$la^syyJ*$U)Kfs+; zXL6_0RYvS)Kd&}vYczVe>D}c|7C8%Zr=oIZ`Hs7EWuB{R$+WZoZVzwPcjCfCFUK&W zjp=W?R9}VAT9k8w9ZPN8FX!sBb&y|P!M*TeUoQs^Im53Xoh41?z9X*vTj7>G9Zi4V zHjAm_L++ne$Lh09nQB`F_xd;ab&S<3?+|D()q6j6l-_@yuJ*&#YwP3BgYJa^cg$ME z#v=UULVmT&BG;;3LawEMlyUppM^=beS;n7VKIyvGN;hxP9u zqbs(5C0|@0*Kl9`G5T$zzZ%b8U|fXPL6$Fn74#k}pVoy5o3GZp%b&EpBCBU)uc&-O zp?pUzPixo$*V=PU+o-L)5b{6fvS#{=lG5*RC7)-~51Q1HFZLX~(wEcTZAVOBNBU;c zPuuikDgIzT=BU~ z_du5Szj3Kd`um^cPzcRKZ$8eB(GKpHbB)Rx2hcgLkqd#~9dO`lD#vb+%L@AUT?*-2Jx`g)r@JmwC0ulW-1*528{ zCi0eZ?Ytl=t9$$F{XPuz*n6)Dh~s@b(mSA|bdh3wQ>O@*P2qPlvXuxY@`vqd37mJ>UuUzMiShY<~S^*t2oglYi9a59ByL z$RF&5OQE?!mFEOG;kGJkwW-GL7?)qZYwL{eM#f;F zY;pc5%<{ff4Ug*5cJ9%}B}oHY$f(V8l=YyWd`<19j3tpMv>q4ejgD#i?@R}^`JUyd zP1QMLpW*F`Q~M0BDD_zz*Un3f3{>1c|emOst-KeqJNFg+be~EVY#=pi{IakR` z4R@WdgBQOp(80`pU7^C!evPYlExamw$d2IwK6@cX?sQ#~p0si;+q{2^p7=Vy@G?`! zd)xkV9rv#NwSJ6o^}gKX*PQExkj*Y^7l`SvyaL-_y<{xs^0|UF_x5hTzjhAZ`{z#R zC}76cpZ`(7njG`MlQecN@yFk6r>R#kHp1)HjKA?SJnC1~f3M5rsW-Q!%M+3LoJE)$ z<^{#(@y6<_O+MMty`b;u9`yaE#2i~;l0;v1JiS!@(3tpSo^9i9?w51<^_uYaHP>9= zxADRI`b164OR`)+yi3OH_v6Nnue0uJOnp0%#|v>|hy8A9%$sC!oC7?PZk1Zj#iU6~ z+aGuylJ`{FF#RZ`>cgfd=60f)>Sx~f3X}ehbJebx^?1_0yK4TATkFhMYKkCCHvs7{ zUB*3HBK@AJW6YTPLc;cUBT34+7GG=Hx`WRO?#-|E`#ZV^LO1=F8CoG6@%Am!pEb+1 z?=iu3dBC+0(epwbzTuY9PcvbYr=4q-Yt`Cqg5T0?33z4>@#XK*XrzfMzF)vF+_I%N z+WEBy|F(krv=sj#zKhPJ9N8KCI{y)OJC~Iy=DsP`tM4T8R4yC6@W;1%OPAjV8;8k* zl9DFAzH#HhpTFvQOuzS7S+uVD_Ra9VUXQD|nz%-6p2OxmrSIpeeZ48e*F(yP_17r< zG*`t%CVdy5z0jOGs~5cX@j!aA&Q<#Z({BFR3HYGO~V^JL~tQ(tdSaEsw@ z_JrXNb^*VQ=-7JS1}yoz>BqrdBY&NrzB5%{25I1*^4lXc4?oDU<5Fdna}8Z!xQ5_W zaDVqIUnjE98T!{nF;ju|JDP*$yA7x8EDG*-0#3fbR1`hS5%aw!@yv5If6ugw{L2dN zbpg)~`@$5@)p_IjnzFcSG}QR{n#d!YZ=;d5Uem@H<&G z_cRW+zi)J@&!%CFMfzTQWPjb!xKbUL?YuIV$`kV*Hkk1qro6*kJGRW))H*zI;JT8~ zZ}*ZUI?a0Pp5~!7I}iEolU|2HZ9n}=G#u+pMN9?a+NYLBYM;&@n)a!%W6fzhZYHvp z>UXrcV?;B2HGp)t-L_LOAGJl>mLcFv8n%QY%Y|?JM_&SNLPEC;Qityh{x-0QWWbl&fG_dNxGA06V#;EjqcX-)1;ge2#*E<1qhNIb9m# zA1iE(y-isPadVp11=V-;QX@;;x>u-ufLSF%uZxk7acy?L-`h5;g}*f&fi&GYA;^2LbVuu`^riH>W=(eeHR3KtM8d^q_Zg4NpS{qTdM`jd%xupC$+pW?`5}|OJv4qI z^P|ov_5{bw>47aKF>$>tEzO*84qK|E(#s460Lo{adrTFlDH)t7PM9@LSt+NQd6X z(wGfV6R^xxZu@=E=f5Be{}Jphntxg?U$CD!GvLddKlGhoe@~NT57mlXz3^5Hn|@mG z(Htxt}s}-IMinkbLF$Fkb_C=2h;$nDR<(JFp%#hu0(h-|*&8?=b79 zqRP*eByaLM!_!6F3n4#?_dg7MV~c%U1*+|9Y`$)LKfHfP{Xg4=V@lsm`mxX1ev3(u zCI#!i!G^w>fv%^;Z2;&Xu(Ns`gWGKhf&X4F}GNzIgs7 zN`26y%kp(HA1b&P1$>$NNAKr0q8muf)pK0Kc5l(@^PC&-Wcj}<%zkPAcE$WUY4kXv z_@y5MpEhk7ocBNbFn*LLgMTvPp619t_(wl$_~n;SaL)+wXE(I=Fh`EW&5?C*$VM?? zb*#^0h4gUgi=z53m%QN5e?eJdz>{R8A zEsgEPS(C4Vx_E(nkujR~TPGST(#@rLldsm~(q1!o=c8x*W0T&y={9LrY*iC-zNgMyw@D2O&wH(5SUPOJDg)zVE#9SLqzwv4cfHq-jD{(;{TSz~UHX=hLGz3kvb_K6netp z*kzE_pZx7_j>XJ#zyHs)d2VsRaBB~w;9d~4d6qU|mlD+j44u&%lDVI&``-<>ej`W0 zJtN>w8)uv0aTYxoMLxB|GFRhGhHrq+3hwg)zDRq7{rbLs6;Zz-o8NCo_tYSNmb%KA z5zK8_-!)ZV&2e>$M!p$OI^_G^^L+hHt8e|G_0?F~wS2PEDY!Rf;X?yc^ST*Nn&2IU| zZBhNtG{1Xesf2l8!j8c{U+%k}8{bc;da7MB)Y9YYgwG$-+UqE|H)i2G#3mv=nuTxQ z$~W!z+Z9>(^zTCs5Kw%{B>l8x`e}l=f_rH;zQ#lOTH)Kad_jGmmxb>L^__&T{M$yp zAtN69+bn!I7!Ihe*B|;Fqc+P|<+tBm&x!9}9icpcPH0=Ao0b z@TJW|s^1{_8`9;uvnjqm4v|OiS^DkQHp|ykq}oxxYXiQlXBWNiO(wPq($6YtmmoQ- z_sEJUvEggPN{g$uyN=(A&<|w~0|1A5WCOxlm8P~rs<@7Ly6x=HU zc}EV}e+_VEm^9HM{P#e$?=bxPcNqS9<~RlSx`03429}s$-<x_iqg6DxbZ8eS4Ioc=UL+rJEv0z1U zF9^nDbPmgH$U5o7^-C3eD!=^?reBsUKW!NBhR=Q}eIwVt`$zBhp`|Nc2&E5}bsogj z?FYA{)ve?%Q+Am@j&C^IpTCamFP0qYw@u5{=Ly9axH8}h+4YzR)9iXpq@U(0+cC0q z`LdiBFde�!K4n!q z$X-|aN5j|e^PL*-W$rcEhlF3XVz!#z+NnKqo#mBID!8{l+t;0_oH1+XIQbiF{!Wvf zIWtTCL)K2U+ctb{>GeE63tw-zrQ-L#b~mT$qA?-5z}^wV7j?6oA{Fnqg~Z{6p+ z`>go(I)qRCx(?sKpG?1M>?pXm1$>$PiinYZ?T^NlU%WMpl~v2Tr<(-#$^&@ivy2c& zqslK{{eOhsyVK-7?*Lw0mJ!;%Xx!)Bgm=yI`hF7k#DF)`Mz0^w3Y+NZ{%};h>aRN1 zow0o*?}}~`+}}0&{S`kSWxP_Gd=0=be5HRje9~0~_sap_O6K<~5)I51-1p}pa}Hn? zp83BTo<5)F%7AC$kiELHBaZ`ZDZEiJTV){1Lqhe%-FuaJE&GaI|^NDkSWwk=P{mk?<;^G?5%AxDj5A%Dd&QxDmhy}-G$uuaVK z!}g>sz{=N8+U6pYuhM1;SihopC*bY0V<_Og`ETQmo7-wBOJkwyI`ghFz-KSS*s#;T z%dhs6tb=y!&6=Egf9gnoCyV>d2<0E;+WQ;wuadv}zNY=}P5H7JZQRTJRg|s0bn9`Z ze6^()LhmwIY|XBU2TDIk`t~EHSNmvgTeRsl$9o~f@6A6!A=*Ow1>Uk+L*iZXFWLOv zHhXwZ_1-LvguXezTTA2<*thvKhP`mWce%a)suMN^{N$%`bL<1YW#`y>2+O%@uQht- z_nmwEGyJhLkmfUF@Nvs5`<|?756z`hmP6~7f_vN3eU6#TJxS*OdQdx0bM@ZG^keWY zaC5*Dou}A-rMAjuo!=(^yw&3lJ}bCaWyv4uugtxB+&Ha=Pvdm5)X3P*XD_7d0(;Dq zY^l#mtM*d9k}|`gw(~+dM{a#8V_2d5S&@1Vw&aO^cz6GD}oNRn-k`LkQg|FlUBVW*dr)J?hLi_3LYNO>F^5whxDe?Vr zh<{0XUXsMFVEM-UarB*lFUvUEI+CKRJ$vCVd4Q?E_RI?IhXekoKI%B}A?>*c->T&c z=8^NW@f|#$O1=`k^FPqY7x0~O@Az8b>x6I5@&#-39d&;H#Ek31jc@9wd3ak-G;%he z0}AdZ0^az3O7yd3K72+}JCtDqsC}^EFJ(+AxL*$V)B0)W(0=NM&#~*~p2r5?bpcohnRW?{q>IlZ(6-0 z8=2VoqZ7VO%jer9+(b6Mi38hF^`3=q=b=VE*YEe+&&;k*i6;8|;EId4$Yrg6nBi^n z+w-pn@M>RiFs;{5o#K7C;a&H6zaQ{s@r&uzu33l8u~>#T#b)19jJyir^GDKI?PoR~ zY0C27ZP^PQ^bVQK_72$qR~J`frQuuTvlr5QTX~Ind1vt-uGu#F11sctH18~$M;0xQ z_Elc!OxeDIS&0W#TpL`SkG6fmXD?()uX<}9tvl7E_s0VDj*KZY20}(0Zw$yS7VboN z;Ol^|tjh51o6_}88pAi9p55CcPmsUd?nx(7`P0VE!HZib`L^I$sWx)V**OR~;^($Z z8%4%mj-O`8nEz@hry1_@(@i-&DS0FH)o&U4Eu|#tC!Moyv-$e`e4*bXl9|~oU$OSG zv*zqxTe?=x)m>p_pxz4ZO||~G?#Q$U%*Ot9iF9`6QHIBt$-O?{iS_|o^#hhyPZyBU z89c@C1@qS>S@`ttd{cy;F6@zi`7tKHKPKIagZvqNS(v++8cFt?UpE}}v`M$s$u@sI zIzQkU$ZjXfUD11Qnk&!yiMEWpl%o(gzo@@Uxi%he>LR`ILil?WnDl(NrqVZZZQAs0 zCN%>)e%y<%kE`bihQHZQpZ;zc=`%B4HNVu?ntW<+FT{@5v6K?qwpE=JLhpcI_>$F2 z$zRUZXX~Ibsob;-RRLe7oaz6MgQ~~L*$r>AoiAqi ztl*v-@J8D*ci})ey*(ftGc~5)gT4Cm0=|r~WzKaReXm}|INNxV;a~K}+(`ldMEvh% zCq|iMd6x1oY1Q`nACA^18@@W9?}UIab}wLl=TMa4eL(OpjJ)6fkZiSQ8eV;O%xl-X zp6K`gA^v#H>@n_m77LxZ&i1=`s*n7`<@KiA*`zJ^wxHaB1I`x=ah{0hsW$R!q<$TR zx4*&gZnNK0aE}k#Y2c9a1^r2JXYuc8UfPAf;e*C*IOws%e{sMc(Zjg8u$nxYH}*ed zIMfzi2+eKjzNQ8eXsmX>%G7JcHT2K@@cO0_^!^vmpgJGq2!YdKKUHeAAN?7 z-|)%87g<;1>)Q`smF1I;D7YnA{k05VqvZ?6<4IZgjxZj}So;Pn zpZ0_b?wwD_(*M-^fcC5-Z#R4!mQQg7_r@%I?A20qX1&+{u3fkJr%iTjeL?=rebf>5 zdy*rG?Wg($rXPZN_UwQ++7D^-tkx=>`L4D3Fub+?yz_}bzKn64_H90o%A5aP-qFu9JPSV071?>r&n7s?=9_YU z^kXfks+Z*Ihp+0DhHqat3GSrPU8Jw?$A zHjn)-y>%^6l(>DGWY&JEw9EDppS_T==ZXxH&`37PHRd|F#xF2D{#cH;3H7F{O5C39 zIoGDWWA5Abm^Vf8Ui5f>-pbTRnFXWj=;cyyaz&2n{~Z|Xbg_fbK> z6FZi@cNKltJ3N&AjXiwgErM&t_D8^bXSH9~d&L{q_tli8zTfFKb*SXC7qAQX-)ZN? zC-}~|o>g&a+-%xB9enmeNMH0Fd^68C9IN4BzY@IJl;Q8O()B{W>FRfvlB=K8zV$ZW zl5Q%b*(?fm)WFY(HJBYhDn@eEPa$Nx`2A#>UX8xm$ky)9b2pymw{_-tO^j$>iVj%# zrT@~aD%&n?d{%I;3%H_dPS8eXZ?86;=4!C{ca)jnUY;d?%5WI#h!x*Hm)}o*ySwLR z;X7n++YVO~dRg>Z(^gZ&72H#@@TJ*M4%zB@{&Vd|TP^>pKc4P*oZlB2bGorrtff!I zEWDX>v*g!0w)8q9|Dq>7^51?K|Izk-;_rsP>cX`4xjNvF_W!+TAH56oTK;8U{&TbO zN9Jq&V}aDxEA!RG)88uA=b9cn7X0=Op4LyKlz(qZ}M;W@;xHRpY~qe9eJ(?g{@WiY+nYPoodAABK%}W(;Gu;0A&fM1F&$|iLti$|ThtfB46<=!7m)bf-(mSd2 zF>_wdgq`!6;VI{;zu0gE^FqsG{c)CAFXk$LQuS=C(Wti`^upPh&Uq~0jOzX&bB*S% zMXq7XH^^rN_tq+3#wcH8F3nVKjZN8W7c8IFR|WUlfG^A3f}qiWDeNtH+;cbb!EVp zspI<5#;}sZA)Y39+TNPRbKX7Yk-c^;gRK|HK)Y4S*q_Tkxi zyXmjMmw0KwlSR+56GrvilR$j>Ug_354BwV7*V%{hWsDmyXUx3(>W`+)%bk~*vWEOI z^w&rGeR9CQe>?VlLgQZZ^7x?P)P7pQy*1#B&P(@jPoiwi19>~) zD1VRPD`KutaPJ8C1~TPLtU$(n-(b*iOgz)@Yz-TpU>v3DJCezxZ!T~3|_#uFZ-E76_>_D`}<9rZrxM}y&I=q+t~l{NNGynYJ6l< zcFdE$n6LdCQl7`jQ=@!bep%Yzcp-e&Aa^h2e6o*~ubHd%14c%F%&N?WBJ)+}!ra~1 zeA?F(UuE(QTV0LhOY~r)^iMaK0#)7uSLX*!zR+GbRbE1y*J!@DI!iBA=j9I>uFaGT zE3tmPgg$k)kR*T`+w^$xJ>BZg<=FrLE1L^_cyeh108eYN4%IP!x2*J9icz9st8 zslDV){O#as{QA; z-oM0*i!MHUAvBKUJC;}8UrwZFYP0HolV{rG(AaB^e!q;(a&zh%F3K~=mHf1+&q^v^ z=Gv2fo-wi(nT(QKA!KiQ=LhT>uQ{WftIL+P;m^-!RQU7#z#-qu-K2v_6Yk)^yc?^J znvWz?`)5u4dim@H+9llM$LSCouP8EaWiU?z1La}Lp2-xxl552-%>c2Y{;C%PR4lwZUr_Z7&~ zrjW{;{z{EwSS=(N0hi;7>sF7-vpy%K*u$}ya@yh)Q?Cnp75^peXb>%QMEbJ~DVH*i zL>2xWgWmoj(d##Q*14xVtBJ9kx(80BRL|*u#Gt)j5xj1LSDE>&W}r0_fN_J*(p7#{ z^y5a~YJU!u&17`>K2r_!cpq(Cai)?u8RJ6E^}18X25+&X)2dHDF9LUDSnQ+!oMhXF z-AU3sSp!Ni|N6crdYxW+e2r#OG5NR3v;H>V%qtFKiIeaQYHHkZUfHf;hvn>-0q8~k zTk5ZMl<<#jw})HVFALBwQvK$nerDWSVwe;|uQY`9S#j7;PO49i)71IK@+QdlsQj#o z!)}j$>`U*rCHip$xWyylk1m7Tl(d8Mtk&uQr~jeXu6hkdZ<%>U|Civ}Zb0=?&eSsZ zwtQ|lWoyeS=(iT=w>~QNR+7ntxN5>8e;cw|-v(TTq8`Y1s(g&HOs?}99$#i&6K?@{ zi=$HikOSt0*Rd=f{aDU>-rp9ygdeTn`b(KcPK~sq4SEf#*Y0rfk88USC6{Rz`(+Gx zam8yfcy93ufjS8%f*hY=+#8(tt*KL!2L@4kBd{4ji~`s30Z_K$o0y_xmbLcivF zQvWN}=lF<2e@6A!Vr_HMGQXS6OAz~Ku&GCX%mHuou~%nL1@wy!!oGT3^i!@!T>W9~ zQOvD=$Bq`@6n_8J^$YD7fPUrUq9058#q*rU`D#M?ePdq%a2A0x{R_eA1MMWNBY~dP zPI*4wnb$Ba<@=K5kwE8aXU|-U@~&S>`Tk_N=d&4nMO9!Jm(6Jx8o!vPD*L+wcn5wb zc$J7zPQtwnuh%pXFBr*k1ti`S@b>>v@XB8Tz|2g(Gz?dfB9VmIxdAD53nZ=U*>-+|r zJz#LZ9_vXa7j>}TCs0q(IZ{u(nIBd7uGUiTPIKO^o$(q!XC6ckK^$Excs_&2d9t|L zIfp4ndH%Ul-j?jgktOTnomEKKS7bTCMRbTufzz3C?lXY;q8CVgi`dIKsbY0(>H4q( zf<$J0d+{i`OM${NZbosUG;)#DH@Abw`UpLcHlzB-CnL6bHMGX5!@cJmVQh> z(@uY{ueEE2IP^*)@aMIH)8>hTJ==2EKfoER6&%i6C)xI-Y#iC691jD~>-vc7r(&Yz zIv(tI;!u?O3()V-dPkG`_4c_&j`Q0ZiGvMiKvBQsFq|7)Cw7l4so#z$m8stX{Z7?? zHT4fbKXSd)Ke%N5G1vMRpx>_gug3n8!*LGs7ODT$*xv&E4%L4(_76b+!2hED1?b1> zUcLWHj=;V`^q4u2Kc3{HzxwQE1%zYJFM7A=$CLW$aXHw8jbt*Q;$qVf zQXT#s15R6q;G~@U_qg6a$O%T!FDOeogRWqIM2^Bf<8HypH}+Jmj#jWeIXGDIg^vj5 zRUMpz`-yB0_2`F6=q-F+^hR(k?j&4)R~kQX*XHPkj|?AGhp*tziZ=jI2;mOs7k)+b z`;y};0=Jj>i4yF~M_NhtOj2H7aVxrv&l~D$nOdoK(BIWyxiq_W*q18ZUo!SjInBwDY3^y)z_=Oc7fy;k&m5hEeabT5 z_2WGQ&+jO*zf1G6?Ea77ntR5Wf0^fCIl(?)rJRIS9NYl?Ht0uwF8U*;U3E*&-xd!i zaxn78t}`aL(|**`J0tZpC)>ZwcO3&`*cW?!)zP^}UHTf_7kff*+p))V651bJav%ib zhrky>k9bJGbOCqvS@lbDzgq8h8_;=;MLl0llxJto>K&BUcn*zn2_?{m*{Ut0a>t^*%7hp}8xIyX59iM~wO{(C_@9tUnFO_PIP? zi7v37uPlcC0Q6`7D)pB-sQ4$v?!=Z6Y}Pje`FWLZPs(S^pY&G^iy-TZ9*h0Pi&9@j zQa^bg8l%!QlJeD%5B*)_t6k&+qa5N?11EHd*wdM8 zkH~1}o%?dBW@x{#JtfD(w~8}v+T;1#^p%*yX#vTpnenF$IFrYTJ&E(wy7c>B zIef)NdGcJG@q7S!3ir|jY)D~#BIQy6};Duf*ctyZ>Qc7IJ z3X#~yu>M8JA2?0q`921fU-CT8V!^L7@-@lv4?mQfequYOAU~k;Df_ha@sMzyT*o=> zFT`^=Z<2b(Og+|m$NP<`?Wf->p*MM%=*=ee23(%`9N^JvVhqFcY)=RD+b$PSxh#xSP+Vs?9-^lp&TSX@X@$4rn{%{c0v!62eyN3Cc#r4}B>m#L{|d&i z?MdByX+5mJ2RJRQf>WEUKg}=PQyZ^PgnCoZ>w1NHRnUu^3ZJUpLelOY?{(Svkt~#F zuFUj9J@D#pUE2N}xZK!}U*bp-&}ZWO>H|)T;2WP3)Kf z?Ig^v%(&Qm>*F|tZz`CONMD+U?)#|J8IWNg#>v2t^e#_Xb5|8twRKKqfGk7n&xC|cO zZIC*soh3%D*?>2y-(Tp(-%R00l6aYY(v-UDJ3t&;Vc62C-!B+&P{4cVQaCLcIP6>c zsO~Pg*K1-z_}T^RRupI_Qe0_0#RO;XUMJu|?o)bBSi zb~A-ndEmh{sQ`75jRw#j)t_**q5tE*n*Pcdv-0d9_SeZ_>-oMT;>UnLuK0=e-n$$= zP9|OkzSDl-SHD;K!^AwizscC2-XB)`IWD3>%s15^H|;+)p`UT~k;5+-m<3%A59#+N z;LIydo56W$(b$oR16xvIaQ>4@ItXmv2yk=t`zDG0eJFvuWcxTDeIe|HR6k+YJ6?`H zEJ__LfMS1E1Fv84Mv0FUJ}-giFkwA_L7RfXz(9e4j=RIW#F^F_{?INey@k^VG7@rz)$ZloTH?=xB{{R&=2YNdHNk~ z=wFi1&m2dYua9-&I-k2~QkI6>xeQ>UnoRrp2qwc5}SU0B=$8#tq&h3A~u~`y>1K z3`_EqNEB#SQP}6(`+)REr@{G90w=wHc~O`2ScYMo2W`NM#RYH9;9cc~2eXoRj{k?C zzxE@dKV|gan9$Gkf9iWm=x|t-+8=~qJ0p0OFWxP<^<*N2U)b;N|CH^e1DIkX^@vvw zyrBmLubE7w@E1Ms(#Kue;!ZurcRsTizFf>H%Ow}|t_0rFd%)m87$*X0yKCOUV;lFCouWUV$6Kk#niLsRoBy-S-I3xf-$%E zOVS>5&nbMuKfj(h>S=q#^4nv7HvWI5e#1`N-ETfT16Zx!^-1*4ZU6jwnov*Cm#y}k z^v|t_^LPUFH2h5JS&JCS6n^|!I6?d_;Lptoeiha(rtoXF2?`)I=URkoC#LYK1iojS z&I$Hn2}qBna+NOYRugbrkFjun{EYGYtHvDxZbgCMCgRb@61dAeOVW??nash+agvW` z`>Kx<+ zd~se}8?(>6F&r~^WvFKW^(_2U{4xfbDST?89ckm-*)Ezf4qsG2b^-ds`m9;R_%)c& z&pZPU^uVWyvlGXo?4MFRJJ>!YcJTYzOyS4p&Af=YKP$gx-r! zHJ%Y{`<399n|TpT;EpUE*Bh{V3Bq`sAwn{>Mb7p=ScGZmQlRbPIG;fMrN5Q>drZ5& z`jqJxk7wWJ5k}H_g|84z=K!1Irz9Wk(`WS(>%g8Qe&%^QE^6yD#}A)b98;Xcx_w~+ zXCY`mq$lr7W?!Ssct)-0k8{zueQxP!5-*?61}a{U!CSQ)9!5?2IjEz_{%Hc< z(DPznkr|hdJ!$+JbGf(O%vg~+a)`rc7UO@ha6bFWaTbBo_JW0T_bbDxfj#ATX0RYE z_u_g@yKYM0c;9mm7Go&eH{Bg~Ip;ia;0_%mxE%)foCNL?@rggqgkm#A5>LoW+*<%{ z$J@p2qf#Btx8_VgxWs?Nq}2G2g(;OJ1pQE1fW6FWu{Uwu@WljfkL&xg<%eS`7{60r zm~CFi%>M2HZfitv&At5aZ3)~FmvOVmOTg4|!|^-^ochBAr@_qY4JkNwym4-11^IsV z5qrU6drR>QV5>fh*lqgfhy>n3&~3e!7s;17F)XEnB#KFV?E>xr?Vmx^!xaAYA58z) z=e1bnGS6%2w<$iOc$28jIw-*TbOOiiS(0}9SrXzz@r+>nNby_Zd}L1o$8Fx?Vj_Ls zt_4mLa2oVk$6;g7h6GN=y>p6CuLpW_w%)PJ>rFwg`88tqi0Pl_elHHO`&#W`eGxuO zc#P;J=JPL9&#h0PQ-e<>^y&*luicEVud3cM-@VQW&SOZVUpsS5IO89hydxug zZvxlN|J}C#%V6IeaAp)|05nthEeV{K)MxCR=j2+FcL9naTNFi{TO)RiQW+_HdO|<# z89mll8%2CQQRL(1y6I4ruTNQv?eCaCw+^2;^s9?SKiBxv=r8}>&<;3sz-zxw{6*l` z>a&TJI8$c|KmI#2&zAd6M<4N*x_+r$gSDYl>|bl_|8)Yt$9;e7d~+9N$>7ZmGUHbt z@Cq*wyl&&qM-q7M?>5T`w&FS~?FDSaWB)7yujL}aYc})a?gXAS9&mp)Wjt`6l%9zD zklU^JlGM)}m(bxn;WLXJZ;*N?jbHX8>YZ6)pWh9iqde)G)OBKv_N)`!A=JYZ-ju*i z+ds*>o3O_=dnmmDN5Aly#aYGaa!@$V6FBL9!LdY&TO7YuL%(dj*pX-ax+f{@0^>wI^ee9s{Ys;Mc~ajz8_lmY$`4)0cSYjR zt5vV6 zZvN+-qu$7WiJ#0h6Q46q%r)nCn4QOxoq0|y>e0u;UlP5zuFcR()R_~^$MRyHfTota z4dvj|hMq4A)?WPWBD)}S z-{pP0WzZSLCq68;_Tz6S!C$FIMq(>2ZIb63j_*t#rHt7l-=Jfb@h*x_^;ZOYUTq5R z`K?(e+|T2J^;ja(kKf$-x(e8=f94y4A5FG-V#$31-}?!P^jR(I2j}b9cLdMu3*p11 z@tm`aOhES85I%dqCwL|J+exW?h;5kZL(0$L^K#|ccU;h5ps zb(z}7dF&)RR^bRFdHYG`3ZK)he1&?203U;1jp}i}ISKyA36%4z`4#;7GwtA>!uVlP zq%e?N-ld!PE_~&|eC(=PFWaa>9G;~YKPLWXUpUESY|C@IvDERMc-I>Hh0p2sC{C%u z_^Ohzq1JRG%JQ{5Wls1nwC2R5&WTZ#4d46g#GJ^ScL=)H88X$_|IPSReO1~wKqgZ7 z+60!h=D^C7j--A+KEtX%gTI-=r+Vq*xFtou2D0<`)M@PCSYisl@GH}n5$}B<&*@N{ zIteuzv)?L!RrfV%7j0w;e=&jUc7B7^4s3%5tUBm)0%zvyf|I8)J^aoD&d4(BKli?n zF`((|zteu;?;8>PeWv}FC-4WBIQQdy6qKi8ivy})cLi*n(74uOaDxfl5s!WBgv)WA zbKS{%0(GNeXWZcZ_?M=idR*<4^SUVPM0wIqvjbv3%>#FQOmI#74d0!>b-A9vrakRy zLdNl0>`{BaD>&W2WeRUf;G})mlJ~ailgBg8Ja6S5SK~^PY2OJ6{mgS+=bi8AC%JBG z3}ZIubB)gZVI4zQ6M7QgbIJ+welBSGNcQjS!|ESMds^|glU%-&>GjU*PAIgi{+Ozx z&cuA?7G0*I#N%Pe6hAKY)hFe9(w=YSmz%(mo|N@&KV)lEuUh?vdX~99Sa7{QC;%4c zZOw$(L7SX}_#PNs?d)k9P@eZgDW{LLJkhSiB^?SlzBs2!jJ#gQz`l&Vi5yo?qZ^VN z6cfNXPp5%XIVtr=d%8Dj=3_=J9C0< zus@T(7RGjC+)e#f>fs()?N9Da19N5tGUhpZTAndYKOuU9ss{T~^y031d=E?c??f-2 z)Jwa@aGne|dsoz22Du989mv$n=&u|+Ad%8lj?HXmJM?RQFZDXUW!-7^?sv6!7<$!n zqE}BHCTkzW?*pfp>zpg{x7ylNZ(g*hmmY6Y--(ilJG_7~QAiXyo-0Ba-%pAi{|vl+ zz#CA!Sx@^aVT#-~Hx^YKoM~SJ^!Yx|ia&~di^*|>F}}o^0`G>q%z|JGR)!3|#1;N- z(d1K7PhPSfmvMm2Wpc`*5!LB5Z07v!23~w#@XY-t^pnSVTDyIE#;Srcr4gl)# z83u0AKLoc1G0jO8t7qKziNP)`G$>2%=};AafzILy4EwhL_lODt2b^~crS22b@0;O~ z1;!yFB?FNYXX~I}92EUf(vS6-Zk*+J5>mX#Z#1XU5?}g(Hxm}Ty5u;{+~);gQ|dM^ zhi@E6Wym$u0pQIaBzWA5I%(PaY=LfGb7Wi|I6wC7!Z)zUoAJGkfn2Gt1b;iJVztM< zK3L;C7oFB$vhN#&Zu}_0sa6=wo3!;Sr8gXjV*ESL`c<#e=v61r$QYH)J0P6?$DV1w z%BP&E#nSed&X=sgl3|tDhq&#liOGF+PrlgRV{Cx!8DoNdm_7uYy-h82>TBc@CM9N6 zr;ka*i|V9BthWoFDbDFiMA8N&G{k*Y~nYoS)UK%bMbhQ zXWdS6iP_x4^5;I1(;GWj2$w^@@&wTzRCTwQFJ~Co<0EkJcsUh^c60z|P;uIlIB9!2 zXYE1ku|lk~cJb_HUiC}xx077_=Dd0;3E+saubEQLM4moP&qNq!qxh^nTH0v#3*p-y zH`k}`=Y-Bn;H^mk9Qv#UpCTPwiR-}&lX#h94EsuJj1gxDpGxiPA#HQ`-~>))yb1Dg zp7i*_xX&>#bd0nwao_QOeqh=+;C)UK(i+^Ezwo9w5= zH5OlOnd7|7KY5tLKB&fLXpQJaazQXHcP$tkz-@)hYZl`5;L~=7;Eg9^+|t*U!Nsr} z9(}lg&;IiSZwC7|CoLPJgWXt{J%cg(xDr0vdy(L#yw@>pUf>cR-B5zXEm+Gxk`YDB zV0*iO(^D=u`;&fm-Ag);emMu0DdX34EU?&*Gr*`=FL+f+yp(G@*u1YB z+aP)=^Ss5xC&slF4epE!+%Ju(e9ARJ<{6vwQ;jKSY-(o*^!qNB`pmOGj8Ca?IM~2T zbM>0)jX-a)a%EG$+Sv3|HvMXR)oN?fwEEA^``|sXb!I?|;uj?A?{K`P;tvKM$*#}Qt$HcGuIXPQfn zAKo>xhY72uJ$!e;wBm6u<0O|jj_V8Bpx9k@~2 zH}$gcV3`LV`)L7q?M;H0xMs8Z2?%NNtOj~T*n9P=9?w^u#P@42Jz&DP5Qj`MexyG8 z;`BdoXKxaF`VW?TN_4lM*lXBpJBdw5rDTUsF7O6KMbID1(_O8yV^k-cf$hs_;b$KYODl6-X z?5rzSWnC2x`c|#>@zJ8}%d#)euFAe5`^xOA95s&P4&XK4A$UB8agxV8kVpQb*yCo# z4(bta3V2iR61;|FfAuW=okD~oNZ7qaKl8b#{PzjoEGuDh>!;z&ekz72e{X46^#_yj zCbhF>IDzTP3DfJ9LIF6xZG^ChzPC&2a1ER?u>`T#uvdu@6u78zqeF=pUB6O@+t2xY~l=XcFL4*fqd61 zkne%~%qx)Ry9-M0fBE+K@UTSe708!Be&S`yvp?z~AAKOhpGp7l2PV}&IC1Dt5YVN+ zdZ0h7`fmHbbdG0AikJoPL zdQSgCuk<4sdL?GK1PghkCl^Xi|3kjwqat6CjOR&tex*s}>AxPxw{?sB3}`1Y?wSgC zufxNCRS=$me$j)XZ{9V5@f=(7nT6m4UU2}C^g!VDOv(vK9w8fID)n_ zg*SiQoKMsnlci+k-xUw6cwof?D;`+!z={V}Jh0+{6%VXTYYim-=zFQaQ_pp_RHXB zgO69(W$>#!`r1-0W;=rSE0O}Eq5I}&<)&eP@};ZgrM?x()VMSjeK&-=V*LO%2d>8E=9%`^l) z{ciJ#&vx7VZ!5o1ac&?8(rVm4V+(EmvEbuX#u@y71rFuM6lW{`X4;$U&VLX1#Ifa% zLv!44`1>Agmo5J~@YlHTOTjO4;jaV#Y#07ERO(jmpojh6^sxUi@K15UKSgm)kFpk@ z53j#-Qso+<&29KfE$7%tO(jaed`Ulq!;$J$o^z1N#=Bk1(QQt${dc{#j_~3nJFc0y z`eBvximS`vn7H~?#bJzL`jygVp4j!T#MS>h`_;Fdlpe2kEB*gEu1=^OCa%6<*|`!| z-S>MdaaH1riK_<}%J}w*t3_J=a^q^b%6rAt>$H5ixcaXu=M`6%!!dF7lZs>F>bI3% ziK{XWT6Ld$yO6%8@I1DR2w|9gvB~55#X7Hj_@5NdeBJOe!0=pQ?BrBbtt=(8_5K58 z9CH(@hu23t=^3TXxH#+-!HsIUjq?Za8LMsnGamfEfzP(u^89{;8~=lP{b9?;!KeK* zYM+^3e^XknGJL*&C~ewjN7CPnB`O>qe%J zS+rgM;m~6rET|lQrp8IO+!K&H10^;;7?l3bDw29nR(-p_U#sP2-F~yuuTyy&_es>p za}k^WEcp0|tqlG%;B$=F^7smcGmc7i{=7-^>osrE{5H)yZa_yN{Z;4l9S-I5@p_x* zD{g%Dhs}Qk^0de1uZKN6*R%N-gMY3I|1$8|mu~VLH)~H5zh8~NQ-_}Yjufts+xKy z9~V8Z^8IR;k?YZVC-mhqZC1U8Z|aTQCisYM%R4N5b%G1+>wk#*Vpu!rBsck|o99`fY7$@Bc)tslvE zlaF}tj{v{G#r`8b_{VtQ6ez#pDH(w`0+Wg1fd1)MKXNa~WXqkWayGvSvRwac{&w)# zPMhEDA^)JtpK!X246`qIOzpiHk{l#V)-&v((hwDTsVx$fX6-F#yz zXV(A!RJj{eZUfrMy@kJ6>@fSjGnD3DhiO6m6%o=`ujQ;zaFym0%uBnrhRsOKZhRMX7hgm zKKr9s_04?}TW%V1tf5Zjitsnn{YsxAWClLJ7tC=qta4_at9V9SF8m%8kgC5UdG6CP z4q8>=r^vbAapRw*GR#cBiZbF@J-&4c%g|18NSH}kCb>ns`jx>Os0HF_XEOB z@=bZK>U)(hJeO{tp-kRy?Nbb{$@d?XzMdJr$=jBxzh=zTBo-PylNWtg%Ab3kj5FN! za?(GPHrl!B2ireI*Dr0~?9ozg&--fB!f|WxrB?KBiP(WxMmXt6rH$NC7VULC{YAC^ zo0@-1^HS9}-xd5<9f!AReRm`0-2bccTa^DA7{Imd?SB%_nYec*n#c8|OK~q!d6OAG zn|x@Q@w%Y%W4Zffl>zaqJs(Pr5dMVL)1oiN`IXvtjT$zs?RdM=d>0PevGy$Szlq-_ z-rMqL;k<*f?UkN`>|HQDDHR*Pn0(HfFNSaO+-2tT=qIH8@fW0?dTqZp^A*RBqGjqE z;Xy%4GW$%E_bk)j=DNzhPf>wG0LDF=|5gwFC%|`m4qylPJeRO>ZUf(3JJWo79R0K6 zYIDByhaEQt3WU$PnN0th{C^u)TTfUT|NmB89ej;@`(JL{+6j zJ;*snD^#v>t;knte!24Btob#XS8MroO4lg8O6j9Ce~sq#TFz?`rZ1@-=6vHqG!sM0 zch-W~=XEaXDB~Y9YxaIqBK?5ke?tX6t2k{+Ux%D$KX$#h0Els`@L8$X*fR#+8W;Z0 zz~emmn96U*-%Q*`n>{vZK}+U^%dEp@9v}2)(KP$pD6qJ%vH916$2oC}%5TNrOb@|{ zjH^GhaLn@>ju8H_=j-S@)5p7Ap1WPjja=7_2Rl%scM3nYRq`s$4S%oFYc-Fm0meTw z9}qpS@)j*Oa^Y=KZ>!?%)7;cstMaB^tmAqj_F45BzFn_r zr^zcj1=rYR$F(-~vne<0fXRJtPuDlNCNEICjK1NUyv3^D@J(K$n|7N#uKnv(Zv69v;w_vn{&-sH1DYRxp72l7{A|sA7YM&X>B}|0M)PLP-=X=v zn$KxH_bdHz%^%kM2bxc5{zuLKs(H9n>OEHTh}sd={B_DdU-J^>U#|4^ns3uQrg^!_ z-LG`N=3mtO`2FJ9>(!x$qeEzk&F6Ut`Jwkn4@Xtr^oyDQUgdV2D7i^S+?3)K-X*wW zN}D(_qqJ7%v+FhU->ZBtH@-+lu21VV@u148*YNFn&3rNKHF4I|YnRV!y+$tDEbTRM zH)7Rm_@>?};|Hz4)Z40Y&#Syv%Z*%I>#bA%oZ=2?UZ?SBR_V2xS9;XzRc_ZC*&*#U z^&VKJ-U2PJ^{CgY+^)C7s<%}A?A6}91{ohMDmS2c+cx1JP`aW~=q{y))o(GSdo}M? zxianldBrz+%}Up+d|dOk8wGz@X}dl%&dfM4`Eu)3?Ry2+=$m!Q#^*XrES^ zXQ50bEi#WN&ouK*q05zT%Rj36X1{6lL|gXd=i?CQl#8St<~qa1vFly@C)B$h_3nFx z^+q23-?z68^#D(h|0dZad=?{=lld3IcB(=XrDaoVgMREYCsmc8x;8F^-0 z{1zLnLhyS&B;&~*Zwvc`KcwYdnm?#G_WYQ9z#XUb!@~E)B{w)<(0!n>^H6llYBQ)( za?`GP#WzJ!{lmfa^L`teF?e+M38YB&N~~vAT6@@Ic*OMzu+Os zXEkq7{pHRhzoK$4_dGK9Me%10HZs-Y!+BwnZC^y~Gv%f~O&(Hvy~;a3Z+cQBjGoE& zY56^BSDgY~gZ|_?X7euwpJUVJ{|E~*$GOda5PY7ym}JlU{c3;Z7sbwwb<)4apMO-l zwkQDmqX-{+{9O(n=Y`GZy2*P#HvdZSI8JQ-WoRG5t*>Lf*kQ&`)l1^BZ75^95%Ze& zo(e|9ZoA*clt2G9DIfg0Jtt~ z`;1&3oWOn?`L=ued_NKX`&6%yPC+UJ`#j`5if{b)OQm0Oq;Mb9()%j>o;prcEO8DO znvnXMHE+{Ata-8SS6Y=ec|>WWZ|C`19@V^1aSD|#(cIWGRU!Q{(I)HqHGgv-7e>BB z{}3sRms@4$RWs@C*HZOs>5rt`_hZR_ruCX}^L4G)^cT15W-VcrraqH5d{6N#d%Vi+ zc^XkWqCXX!3oaHvyjJl}`@0~Sb?*cml)MIIcPZb@-;aaG_+j%u3LX(>ej)hwe5$$& z6Ygb{mrRSkZGYa&)c3ky_oTMtM#l=gFB&=Rt^S9!+pNQ11CRGDZ2lwQU53wREx*}v z-+~PN_7jym8Gkd`a^C?jiqDHGcOw2~vgJMn9%E_2o20+iPzcGE^ZVt#qRrnAS?XV; z`lnJE$(G}DC?-aUg!~J(lVU%Yei?Z>J^!Bi&+IR%Asxl%Jjc5@&vUfpteophpW4AO z#PkuB+bg7V2G9GWqkolpyvBiwCo*HFIqz>(eXntNv|s#b?Arw-t_KHQBK#8-*_OKr zeEQSoe;m9w;&Zy{ufg9;w%q5yWBaO9&dfhs?!%B_J-4bH|7Nn~z6u`keEi}!QoZI= znipzr>jmI2_J_@1rTjb8KTQrQ-q(wq@!jUrPB;EA`1GVL&-q3Eno9Ar{EKIam6qt5 zsW;|<_XPNin|3{XywlCDhduaT^x%KngZ~2${xtY*?f#_)|0NGTZ@;_Y-0s2Wu{7mh zx>P)BuHT=q#`!^r>-5W;0wQPbXK`IR%7wofydoF=b>MMs**G5nkH|Lvli;!5i!YOU zd5?@~v(m!N)H_uDW8<0oa!t$V|J6hPH28}?x>@zjeMh?O2@R`3zYtb_Qy7*cfoG@$mX98yH5lEE2>YQGTCxx zY5PtKif0+On4!|CfM&tP9Q;z<0y{fd_vQe73jyav7iIx+&qW0*Q=(JCKQ`)<&3t~2z%t(SX+)EiYfTaWR@O|Mw>%>J=LX+gv9 zSXuS+x`%P1h%cf*s?vN)^99YFLi}U=WAe^hMDOTqso{1NJRLdXjLqK)9_QB;YS%sZ zo9P3p&;2OVIrz{n|0=;X>%(c_y~c&lxx;y4%hL|tf3W$dfJfwl{}lVXG%xs!&=pE= z*ZNx^%VeHc{JHXX2-`HfFYhO~niQ`*!-(vIM z4?g4R*#`*@{mOKY#>X=L06LNx2d2Nw_2hETLH@^8g3B1jbidYP*3yRkR$b*@o)inSk##E0F_dRP@VZlTCgO;25 zZN^Wr;xL9W*>W6XM7H^7gU1+mvFd*Ve=}X3D|VQAh)X+cKG#{c!{#3c9^O7=LA|FSuK3CKQQOVXQ}*&D)$bRGwaKz{9@k|mi|w{=RA6sRo|6| zNWCi{Ci6*W?W zi9`1HQr>`ixE{8tT(N`V^98RKJ&r#nTklHfaV;rTImQU4S*5+~S$&P*a{MwK7ZLhq zA$@N{&UTlpoWZ$M>DQ~=pwgTJOfM?UeFBq>%l;(p8P`gErXO}FeIAvNdX?r_XS&z2 z&pg-QKTL3HMa;*qb#mU=e16T6ao*A4yoxpa|1na!^NkK<>e=`|B4BsQ(Kd?*64d3KrQ|Y~9 z_$HrM|2?Pn)H~=t-^bBT&K;Y-4fC4grY28X@)ni<9QfpXOxtPpTejSrU=G`9^XpaL z=3ll-#+S|i4w`$C3r-I33tjjxVqK=X|pHiyr(_aUIC@$d-RS_-;7odGNP+ z;CvVLy4kr``JdByW5!J_{8j>cZ2q^wI~AV+?I(^erp@Yymy~a=V@=$$=il#PC-*Qm z|NY=|&9?aug3ozp^9RA_Jh1ue(5{nR_!oHad40pR+s3K$;6IG^Qoia)8Ly_jzry&& z{~0(1kUtH4lgvCZ`5=GXIwcvt$?J4JJ*f4arLylv&UV^-#y8ey^Sf34am6v~?wznN ziZa@sn(k?k_8GoSmu?ojxvyg~_Ky4(jzg+=izqzhC9;IKCalg|N#W-{!gZ1L`-<0VeMMcpYT(%fTa#&1d{# zoU-}cC%EC<2>u!udG2GmuGu&(;Bo%geC~5j#Ha8m=}%+-m%-!svH5-IaLzZI&*u!- zcQ*fr;Bg(d`R73X3>QAnXE{D?`C9O~F4_E@;IDDv^PHY>#Fl>p_-^fb5qvlNzbQ_Y z`p4|k__`O`>Bc|Fh5vo?iV zRn_C%VA`biU3jDDnf<{1N}nfUzF(*wZDtzA{N;LS^LgKmZTY)pAIB}{kx6!3%6l6C zk@6n{08;d5v4^prsS0*d$L7BeeA<+&@iB_OnM}MZQQXcT7BHlHwx#28jHV{BBKf|P zZfm_jbxtyQ*U5rA>k)4|G|rkj?D$=!>y0Tlb|-SJ-_&R1YTqvHI9M55l{RusO7jek z=|=S@$0Cz?4#(z~pgB>Le_i!W{IccFh8+9*PIHlufzKFa6uOV&zJs$jz zdf+_jADy3cf<(p)Q~^SWOPUG*Eu8c$MD&TcvkIG5wa6Y88@n6372d1Bs9?^1RpI1CAwBnr|4@;LR zpEz3D+oJi9=A(x87^it!UUj7CHykB-^iCPqqiVm2kIS_$SKFVf?YHHBsd(I@FhzEX zC(jVlcahTPD!p526VH|HE7w2FI4ailY&InJ8#z;M@=={{?I(%c(>k8ls=_vO6!%G{ zzYNad^L)q2c$dK3U_Ao=4K!R0k1(~0>)pD3j7UgVs6Hopfv z#t57LY4A9=|Dd>6<8P*m754%mo#*ln!`~O(EcG#_GR0JmW1Z=j_^|$Fl{<}0BwOxh z;8E@&)i>958^L1@)5FOPB*E2ycS}* z2@`=WiK(37?^d~!94YB*M&JpRGw0m*s~qRt+*UNOphr21|fZiA0zd0+%lQ|u28;-AM7{As^?YT%s*R>dvopq zFKbP2&+Uq5`fvwiIGzVBK4TOyY(6n*!!s&x{P~~o3w?RYu~MH|r+)(;=lWLVufyL= z*W$xDX7jgt@ZS$U`*`L$!T*e2PkdGL1?Bs$m-6o`e@62^YyJ<-4`~v)6E#0m^Yb;o zQuD2v@76q~`G++BvgXffp0izW*Jyr*=C9X0qV|<5eXZt=n(xs(ruo!ug11lUhcu5W z&exRwvF3A{Kd1RYdYy2T=4&)B)_g|s%N6fgU1tKjq~60ck7|Cl<{LD>O7m*fuT^@h z<~uZhr~3U4rSH=GLz?$${w2*vHUGKhzt?r<0e%j~q_hBQ#me_oAzryD8>vL}M|Lpq^UhVyh z`q{+oKf!j!3!87^x6SW?9~h5pelPfpcQ*ec;ByaR^LdVHVl|30*1O#Qk$vHs0vRV} zT=YSf>y&+7`)cpUvF_9avz(O67c06XQ$BzCDwopaezipHH$q0ww-viz;F8o&Tcuy@TMN;v)ZF;JMlJKJYJf zk-r-}?gi|6$>(0*=06DDxi0*7fya4d%X2?{vJ3w%<$qlJ*UWeBmw3Hm^J$A4|8x)j z<>0S&!MPs1A{RdW!~K(8@9`Li+{@Ygv%qJZvH5GkzXbd~?y7$gJIpve={TXI zT9LVrUZH&UInx{r%I92R zGWSEjsC>p)CcXxqvF4F~5&QETuFv%n)kzuANT1o+(l*zJABgU_$~Q~pZr-;?k+(}(e4+_3qNfJggn{*VX%%O3n}^@EMW z=h)rw?|_{g&nQky{<~#dny4OD+Lnu`mf5G*KWa3j!0bOvJ~b@myVWn;Ycl-?AKGX0 zpH%(_wSM}JsYmIkL*ahA(wwuY$=GG`7Rx@vH@PV!Z|mEBtyOt+Pg*kTkFhu0ApWesQSw}U4@8&x*Q|$JR=IDp`48pDIJNmh;B!s8 z?;dGkm0HoSxMtm{!@i9DHm!Wt!PE*S`@)T1ow!(J;2~x z`|bP1ZsR{&?wyd~Uh$-RMb3 z&sW?jrA-|ByV56W`N{W-zL|gLDQ(K@EkD}j*QvZ|=Psogo0#@0ZT$KPrP+r}c6_?@ zePYip%{w)JSo0~(Pt8KGYJ zzhQl^lF_u^GiC6-@}3V!yG^;`#0=g_zT$xu53G1##RDrISn11la_@xY1)Ry?rcffWy|cwof?D;`+!z={V}Jh0+{ z6%VXTeVCEuw%mThc^<%(_mH-fTII2MXryB-2CcYRA_thG0iqnomKu8->;9>`9Go`V_cWQBb&=TFWzNJX9R^>acfq5G%9Ou-miv?i;=nX9q zA&O$b{7oBTA=Ztu6jfLR^~hWCK`Ymx0W78UWQg@_MeT-I7PYg)tbAI|I_gh#M5U>z z&2^f5oM_Q@BE!N>Fd)2XT}#w0qhn!T4IpbSZplY3&4ndsaF~vj>QkC!Ox?83_>hIp z7(x?JN{FKR$hu7zZ(LuqPTCIP5X$5~dPHFpIXOkN!{N|!2v3gK8m>d1%+OwD@vFBTOe6HSt0pt0kC zxe47#G@V0I4gL+sRy#7RYy*Y`O~r&x;9R?*79&%91?8*PU%f823Ih+q6~J;@mh_RM z=Jbf;3TH5H+JH_^u}=I+NoB^uk$Rx!!47SI5r%rr`q=T#j6OK>mUWxfU5%k5gNK6% zDs=LhoMp}|a1?0_XB|c#8qpGUh7X&#%Bd-qOP9tD3D=(JtKWDT?Wn|m;f)nPM z=GNw0H{Y;pSL>eE-CJ6GZ{8g{b|g2_Ro-xWbHnC6t+(8;`G)N~8ya`}ZVq?(cKe{Q zVZ+8PI~#WGxTJAsrTCJ@)(vl3M{YEoTd}2e%f`m$^;`BhkVWaGmsV4y#AaT)^Tu5) zC`~V@+Ol)oEnBuVZfa@Vy8XuOTZv7v@^rC{joV=1rq(UHTUlC}URu-G)|z5abxP^R zE!+2`H?cOgv~kZZJDf(;r*JnlwqCN4)ik9urDZK-Zf@MYd)IEtw2^drXj^0J=IuMs zmd#C#Ti8t~jrm=QcW#k7JKPFy-La*iVfW@e&D$Fs!@80}0={+EEjt}2#GMk38T5SH zc8AwP-j*A7iJSUJ-Ec?aZpZTjq_%EpZr-((rVTkHh?_RU^49HQ(};4VYsV5)!*-`V z6J+4;8yg5QCDfjlEv;LdHt)u8(jhUERLSZ;+}-HF#R4Suv~F>{&_N1~+IpK~W|xwUJ8va$TuC;0*NqKZZYQ&c z%pD2m^f_eXj;%XdSUOj^XA6u=oI zrW~SW_pa8)t&ZA^!?a0?Q!Geo*PiWdn|Hw1&F~>> z=n(d8{JYEfyR``&(17vMx_fg&Bf`usr#Ie^ zh&MOv-nFG+>y|xEqXsgHQ|cYcC`@+$NJbg^tHJdCSVplanMf;147jOOc0(hY&_ut? zq_R1KoyIv+aHrEvbFKnx{bmtZNELYduI-NN7gL!#Zo&La^>-}9{?g8nQseNB%py)v zrwcQST6f=WO5>TOx9l{9JsE}e5a@H4FlG(77h^EA-rnMjgP~N3J&nyb`rzG>bY?3f zpSo>~MI4P7(VHc%U;oq)N zyw)$!xO+S1CZ^35&cxdk@^$}+GP95I-^nS+1UER*{iX_x^__dTDBpZVW9#niTWjd% z+d{ti=NyfX5ViXQS^iDL`=s9&*s>b~`a!>`0zYq(y{@6*+U>1Pmp1J2UAX?1?eO5b z4Kg1tslYbjx=RqQ{$*36?@Kg(?y$0k#jN>2OKEYte=5*@LT_+1I1`G6JDP_BRb>rY zBAmHnGkHA);H@nxigx;^{pI=j&0YRZe|5jVU?9-$-yf)mXY~aJ{q6oCe@|e}> zMyiTN0`XA)p)&_ft{TebfBO%e4-FhQcF~{*qY;PX+3S18s|0i-GarsK2z` zUpC_(4GsmzgY&`t!Sc2F^@~~U{>4Bj&cYdgO(YV|4~6oO_WApQeZfei!(SWD%`Fet zRt^WI{RjNvb{G=MFGAD0{S!HJfx$xu51vFpXP`4s-MhLkYd$g?o;;{;Ro|)}f5W7| zwZq@z&mRbk`oodPU{(?9>F|&GyH6>d4$OsSLNoqB=f6?^h`+ic*d1sK7Zg+!MEU|1 zk=Ae|KYz?W=8r_C{muQTx`M6{N6ZKM;f#JL#{z5j`Ny&j`m5UmBY}cx|Ac?m-<}l@3?uLJj|S#~#o=(Ij*gD|YYzB_ zSLGu$qi>{Z^7111I~ToDUhbgg=1#*YxdjE0a5$WwUm0J8JRA;{<>iMXE$#kr9UPcH z6o{e0;Yep7l%GeUIugmP8eSDaVb7}a#bCqeLG=gx?13iiz^T#?@AmtfCj4{$vfPTo zA%9uLzR+M+9%X7EAm#9Nup=0%Vo%ie1tx+uxpPp9hQss0*l*y+zNtIw@%_ct_* zV6aXH#sYnTz92d}P}Bz4V5JQCL4Rp!eMR$JpdI#zHG8XQP7me+?Wv0gyHM0H0N0LY z$3u($0rd2Apd%Z82^Lj#V(^Rw7BIPL+iGj30|S8rpnLpNp`omnwzahlh$0mu02{yv zhOe`RLz6kVg9ptY8mdC`F}ddhZ4In(0d!5+Nz4`>e4p2 zxD%Ci`Mc1Y4c(Z6eGu=1T~q!6%@mSg^Ie@ooIcFNycHHjG9?$5&ugz^*4tn&>!RYZ!jI{bVWwnR6WQF0MKzJSwokH{< z@{eQQ^1tptM=+k<8yLx+$u8;+^#+TII)dS<=7#E)u|P{rU#M^}FdNF<@9z$_j0BoP zlkhr6+z7^7dsag%A0nVTCK#i)s|n#H7P&yEK>Lq&suvBL*L-K*vU1)xtq{$zl` zH8(e07%oH*3x{*Dav|m8FK}5DL40rPM1=C^kAwyT@vOLi?{HuqbE1uF+kk%t8-aeT z!23d2GZ2m9!IF}Sit0u9B2=2&gSB%mP&u3xDJ}@-wIZ6-PaqaW@~XzO^SX{23AYDI z2Qk9p2mt=UQ2o4r9^-W}xG&I+MY$jGZ9Le^nG@>E9XV(+dm_-zB_Mksue!2$FpyVK z7|GAe>p_?+nF|aBheKT$3m7M@g|oq~{P81a4xZ269~?(8S`6mpwP!707)&874q_U| z5I}$&sOrm}5ADm2AKB=d-5$Z80F4|E6K{00Q`MU^Ebk%=j_p^E(2Oy&XB7IUV?KET^1d z0SjRZ&_I3AL7@Z~(+$R&=;z18w$-KK(~~Y zm34&HR^*N6lr>}11j$_Ndxirwg@t8R1%1JqCeHJ&z%VvHx!va$#;|uOpAA%0l;$G9 zBM1z_)fL5+^|Sumc&K6_SYBS7UmWwVMTGX}=ZDtj<`z^&%hr~aRaWQc7Z%R?>q;>q z@^dRH)*@9`S7S@)q}(F{KK^FFoW4-#@$3B#T_~?P_RTlocP&ufp|t-xDUaof{@Hio zz4Iua*B2`P952i#U8gU;?q4hT`;>kHKk&u!5v8B`H@|~laj4+E?H@vSM}#ij9&pM_ zqe4&MMbWfpdX3Phe_rr<^aaxAJ}LG%FUg0;Mi+@{{ zeyCLXw_EA$R|-9<^uFtbo>h9|Z)Lo+0Eg+NN};RxLnBD*pA!5|r3dlia`tDh(v!av zdRXblZWR2Ypy&^sE#*T>pA#3lA|&N++$i$zPLgA7P# zJs|QC{tyGw!#|Yr_OQ^yk4yQO(r^ES(DnQQ6r}I27rKK#e1dcp#sS+`#vkH9`qb%y zH=^_-lTzMBA*8qLlJYozkOZmc46(OXUo6*rt;kO){TuvX3RSBQ6Zv!ADs-RH2Yw>* zLrTBvG${`qF7n&(BTLjTR(fci)Yp83l;d#GDQ`bg=v?p_FD6kB(>WiP@n-0)criT7 zLvNGv`R|MT0i{pDi{V&ayjjY>y;aI1^+M;OzgS+fMd*e9k@`Ds5c&=ILeDAv_)nxg zg$+_};%m3kr~h8?_idB%X>D)rcA;n61aIo?LYw*7eUs2;K29oq*~!`;%~F2;=M-P* zUsQXw6a6Bkjek0nKCMIQi`^me#$Ta#2>sTp1aI$Np?fY=d8LiN z^8ZE3jlZguUh^iYe^Tk221S4HPLaRsB8xwn5KOpUy z{#T(FJ}h+myM^AjLFj!-Z#qx#dX=8IPiX8(o&I_pKMKMAZ&!NZS)n_XuKt7I#g+c` zqR_ocpEe`%vr0b;KND|W=|^jYuE0Kl=@SnK-Kq4s_+bdjkG)6A1Nfm|(!F;Hef)jm zzaESmraxRS{_8XJMN(e;UMXMuLn$9n`ge~AJ*f2kCkQ>R^f2NJg&tEb;tK68?-jaF{nw@RulI=j;Af@$ZNoy>eO~BK{X^)n zFA4p|7OnrQLKh<*(_Y_*&`nXH`yLhA|9z3K{*KUjF9^M$^sgF)#>tIi@1Ty?qVEd* zjZIR2x6+qiDe{Hilk&Au>7P-hxqi}~X{8IbKXb=L{)IJCKB4rvLxOMUGmet^HKpZT zuZTCN^p74EdQs`8*GT>Sj|=`|=Sumc(xjU{YK<_ekXM8DPq5&A2>tGr?mXiJwmVjgUJ7(Ozi1?QfRXt9C%9T=C_Fa^t{kF zT`l-sPYb=bLFkG<34P43Mc&YV?o@wh`G+)~4nHIETYoD4*}EY0?q_s9Dcukef7kq3 z${!yPyq4#LzV}iopHuqZACd9}rQZ+|f8p}n@z(>`m(qVlN_YHH%_zP0spD-V(UynG-bT)F*ElQjHY)omhKdt+#$eaDXdw1^$3T zoBiXM(%*Q8lvf6%ygys~yI<)q=y>f5O8Lp_#NT~b_n7XyOysAPo;gSCZOM}IPvnVw zX;|oAV;{u!3?3r%As2|fCAdyz>U~hkeb}ck-SZdGH}p@Bmh!eErTns+r2bx|Kc(?> zQ0Wi+O5_U>XP6$lOz8bekG)UuXOEV06W^Nig*Nf6U1<~F245rPyYORMY){)ULcc-d z(LU@um`r>dz;!W`iDx59n|RiGf|Nh}E5Ykl`Y7NKza4Ru=^aOi{nIB3{n7Uc-GhB7 z(?>D>DL;?6#k9Lx^g~5Lzlimh<&o2b_CFx>T3q)r9o4P%D}7zQ;PolJ73&M0w$z=B51-MROdQ|rd`;>m-M}j}0wAp_a@drVX%ziU^me9qg$@qw$EwtIMFDh;J z>%%w)VKV#A*ttSKsrC;ko!F1QUdoT05PK$-Hv9RG^Q7GDS4WjL`_*1t2Qv-*O#HK` z^bt8E=B^NW@oPf&TqX3GUl;lAHw*nR_Sa7Rn}j~+RFRL? z2t5)NdhlwYiwe`}qmNblwEX)qp-Zn3`R6qr3@DukIo4NzeFW3b@I#uUeYHXx|5sio z^abNmUW9W7rZ4wtdz8Km=QPxh)k%5Jw3HkAxdJJ#ew&o%{zmA!%|f5tFY-;;=P_Ni zR>~)~2z}k5LLaz6=)bHM+P78ck18Ef`b?$6N*{@HK-!n9^mJU>A5r?lO6MzW)~^Dk z&*+o+w^r#xhlQ?E`f!cEbxQBh{i>nw!urkn4gF@UQ>pZyw}^eF{5>1f>4V;oPWQu) zsrv82zABY|4ECndNBuRO&e8re^_8H%7++e|{->@Gdau&wHAs77NC z`;?x3i`X};^w9su-uuUQJ^laxdl8DoWEiG~#W3^y{e!S-)naPZFvN4upKW?>d$v7S ztA<5L!eSUguL>bq3_}eoY_BIih| zJk1W-M-Aj99kJ;j=XllWdoy8wj?5G4>ob`bResGS(jEi((?`I|m_GGYaCHdzy&MnK zGr$=aKwn`o_}y}(*E}N(do@?gi{*cp#q!#$*!*2RvGT&qSbpP<*z~W+KLn}jqsx2x z&RBl+kl6g&WdEtsPpAFMxqs9AzQiR1<#&D=o4#pXEZ5x=%P)V4^0mHCB~BZtZ*)>D z-#R^(pQ1nM{1w;5>KlC+I7<7In_@X78~U_-yo`5MzUH^Z`3Lg564wpn%N~f8oA!v6 zCyt1f-@^G->%Xx$w!Cjg$L6p34&!qf?Z5k0jL$^5S5RLUUkk1#Z;ORNT8dppWYp9|jac-YrT9@`3e>paLO?FtUh2QQU# z5LMsC8t{i>VNb(F;B$HY)JpzGF6`5sIUD6^Ze@FGe*IaN&;0uy7MtG9^+c!t{`OdT zpTtqhzKzU(r5AF|=^S60Uwa?rY1Zcno0z{_&aaieF7oVANS}Q%>U$3N2d!l1H)wBp zQNr-o)0cuf$wSs7zik2Jx84u^rc1zj{qDI8tk!Q;{-Ddj{tD-9G23fQaH+jR2B5*I4} zb&>V@n@4`0`|~1lt;A8P{t@yuC!sxR$@;vrj(l$+>UQbs0 zSXEvhdE(dTuThIoA3cB7koEl4L)PZ|3=SGkMcP(7&Y~>BHPFMQ#8eJq7h&b0hft zD7c9{h5N@=^3RW_s4tXkhksI7aK64E0*-Cz<2=c^6 zMdic$ zlLz*1C7*l<^2?i7hJP=*3f%A{c-~V;-$Q;n1Kji!ICG~|Oho>E)D z!}mt|Hu9?Fu&?16$SVuLi#C84+yYK|4!q)@C_kCJ@6V7IkyGCUCq9q#_cPvaA#djX zAms(fyQDo-|75p;>)2jNFM)3;g*-s6@V`e+>hDFU!8^W(_TgX!Q$V_-a+l)W4XR-E@%I1esedDXQtnMU+`dZrHlu| zzsdhDgk0YIH2mRkAIm4d{1rHz{9QiMo5+u6fWzch<|2KRY$-!|nqLV)UQK!8ui#qp z>rSx#_o0_8NB-56&(DKA;WgMhQT9`+KB?pb9)R5SI^@qC2Ky4;0KZww`jemi5$Ovy zLH=z$^}h)oeWIlOP^`V9P?l&{(+yBpki2HLN< z2i$i6+9OJS;5=~4*N|sYf95yfU<>M7yA`}=2>Oz?fxkN!2S-U3N9* z1J}FYj9tMePk}vkHzTjiH-sKDf?{)^{Daqhh*TA0S{lP2hXdgNE zHKcDn0P>|d;Mx@MUTm*s@)K70OAmR?ZWvDmBanW|Dsbmv;9vKGe|H@Y{`N)4lSYC+ zko8B+2X*8d-hzD9k&sV49P)yrz-N8|`5JPK_>;=Nm3;PEat6|`KMLhH9u2NLg?tRS z{Whe}I2Qcq2EXX~w|5Em}kRMow{JFx^Xh`PtyJz~&r}}4M z2jZBtMfyueqrR6@w{pE?1YG!JZ%@vq7&oDcpfiu|=T z;NQ9aW?TfGcn|XTlFws&H@z0}6CZ_qA$eCP?IDl98C-J-(qDEp()V2o-jDvCvk*M~ z6v#E7a4qupQtlMvmHnDOxHwk6?!#ENrpBg!{=!(c9S>e}8OpC4AIoKj$MVY^;G)Zs z-kFT@lCJFbzZ23-f%>q{$HuP?dRL$244cC!6ll$Ti#`Onv&{;fBF zSDyghbR*cu^Mr*rfg5+_{7HW5P^52YfP5{_hcrL`6WU+%ZamM-yBX;x^ZcT28F?b~ zuf83;k^a2=4siNiD6f4v_$1En3;zkex(nrXt^lWe1Ma>P9A*5JeK)v5;(s+>Qtttu zuo~%e$WtzaypFu#P}rAxFVep>7V;kQ!^a@MZ4KnXC%~)j2kZH-cr94P8>;*k^5F~7 zzpe)$*ZY$?vR?1I$W1}y&t8Z0a*ri z@VZ;TfycpjPeys0$UdGQ-lXudFTVMr+FX9fBy*FkOa;+ z7`%r3>7C$=VUXXw7kCu;4W8H4?gx1u=eJ(+){kLNOETmaa(!vqADkfZt+Kb9e8Zzi zZ#w|;pJo54CY$Y{%IrY_&GZT z>2>_9S;x<{yr0cX2kZBHx00u%LEq*hA>SzX7fOG626*+surKjw@OS$|UVjW&od>D; zq=Edzp5VM=Ay@04(lp2`oYJ6A9r&;Cr@61^o_?uZh9Q$Z#9F1qrnLl@DFUC zL~_Fskf+!o&prt4lS*FwBJ>9wkRO^4Uhf1i;Q4!k3%r%_c@Oyu_NU1Wd5Hd9K)(5U z+E4D;9qrL}0@8=uAh(?eeu(?I+)?1y$3b3r5_n!J>eF=!_}*I~PZ$kO<@rQ4xo8gB zw}yN}0ou2g9I?Tkh2(WFL*F9uVJ@_n=B5)-egowT*q${xDF11Rhm8K6051FnJor@b zI<8kOdEh-6zlJA+tzqa-p9+3d<`d-~Tgm6i{$25=Ga>)#a>zH6k17Ipl9#{9_9CBl z1~~03|F%ksJG9PpeLUCfkQRCmEc4KJUK7~Dr9^_A(2DytInhSX@x%U!qGr2?dU#dQxZ8 zf{-7W4sIZSI|%uwhaeyRH01d+z~3DWd2TVd@I}a1kwecieF@}CryzYF`QGaxuekv7 zrWvSD%1rRP3y^*hxhfa(v@qm*{f7JnubTQ4nq2(1>jv= z&{uy6csT1GLiFeuXWj<>O^=`3!TVnW`T@Bv*9>{yvf@{AfKSX&Y>fhcB zZj$+0+1vgkc*1UoM+Q9)o_8qhOMVsHcOKFgy$L?v0?ytHE@k|f@(Fkv}KqbzhykQd{^*gha>%<-N0Xuf&Hm_ zfLEu3lZJrbY=^%5y}%oCA*7F7tmfgCG@j>cc775ozjrA(Jq7wN zx(M7w?&kT5=BtlI`Ku2?`jhkHB@T!D z@~6ZajFzS;(J}2X6kJ`6q$9Pk=ly8NB={$ZHC~hh}2DB+LTubrIxg zmEcB>ugp=4-Q`s-oFC&Rg=3t;7; zzdjuDT5{4Wkf*Ol`i7N|hsox9!I38+KP(@d+yZ`>=TkM$GQZSUmET03dpz{JHb9=w z{rD(yz4V7lKd2S*w{D4)V(+o>cP2=fQh=VNWl41JB<^y#V=rCqUm4 z@}rFZTVI5H-fd{ll$XH^C4N)-0~^7+aXnns4t|#BTRE?T*Ydn+GkMf>v&`92b# zD1ANTBWuB<-iCbrEO1)~_^fA8pN-`F-ykn~5Avba^q=>^AMb-bYA3rUvB~kiUKl>9aqE{G1!WE#%-M$lve@9`=5yF%`w~3#7Rd9z0>5<@+NYxjTyiqxTfPP7 z-3@v2ci_8Rtlw7fzb*kc{RpmRJf}IcBUZjv;xW~qn|1nB=^bha@??GRH+`Jb3w`5ny50vvbmA;Ye zxsm-l2=dqXK0@vu;E6}0`~va{#xuEtAs@C5_H+#aKby()dx9_92lD#8!0F_KMDUAy zLq3S?C3lebN`kz2DAJ!c1f0G%_+!pro#fLlK>5p)ApceF?^JzrhJjDoiuP$s2Co|m zeVO}%9rM9?i#2(zla0Qr<}($@||` zDbT-#@ksa}@Jt`<8+0hxE9YOzAG^uVP33$u9P%nTe^>SIISl-X#4m~$9swR*j`pb= z3BHp1w`THOSr3f-M>74tk=|wkTi*j0Tfon8JPvYz-Mr6=IKkiV!TFvXeFXKXBZuXD zQProO?Bjm6_ypv4_91=RiQv=Zyh5d~BOf{m?VC0V@&`HJWRSPM40}hN1o@EHQNNUv z!KESA=M>7h{zu8X$f#50)sj2&kiK^e(%;7ORogi5Yq!9jn(<&K;B}4A*P9D2 zxD%W|5qyoTZ>s%QlY@4Y*GOK>@zG3P`wYrkeH!vVC;JnXKOql1UgAS#|H9M3W50wv zFbTX;;t7?$n*7Y?D8GUH!@LO+dpR- z_~?^R-V(C&K=^B0CFIwi1TLHd{@3oXXJa*Zy{sq7{_t`#{WE(7_(0k3D|zu+u)fb- z{{VQ!a_Da*tH0Y)>D$Sd*MT>a@4FJ*P0qU%yp??Rh2Vrn=+B=69!&my7C4hUGz_+p zi`Rk+$rr5xSCc)_Ev${lg}&%ZzLab zKX^0w{*~Y!@`_u*edGn#gOk^xJW!2d9&hYr!V+qjSNd$Qe;^E_p&3IG>!j z20Weo%nW!guSk(z!SOO?ILF%hw`RB4f)t!a2NTbCX~0j1@gNmL05U-Tt# z+SibGz6}0K&R3PZ`W5hXGmw5Y`JE$Beh2xN6CqE173r7k3waTFlmqfc^1*Umt@N+{ z7t$xme5|;r9Xz)b=`|0(3LKz(!aDFWvV878^pQ)ihrYJgpzoey=6@Z$`g!oiH^2dz zkJWsZya`+rLH_o)z)ONi-$`D>cq8#`$je5cJ{jb3AHZKt^ABcZREypO~i zs{XYd(AV`dGO|5{ywt)``g6#Am1wo`P0aQ#$kMh$q#lz z-uwa5-}f8ahkRKz^l$zG@-d6iUa8-J6QzBWeQDoD;fH z_6N(q17?(OI|S^T3jKAd;B$GN**XGzH_!jK9u2Lb@3gY?DZ6W)h?)nx6@b>yR4kbV_;^+||icCbFr zZz9j#27O%)$aTC`=mal05AquF>34&R+>qM>urD$Wd~y!tqw>M8L|{+qnc#&cwCAF; z!4WUyHRpo&vO!-Uqr6PTi2mFr2qpE$P|0#K* z59yzj^t0vB_==p>u&MEd$-$lv-M`U*?HyKp`4AfF`r5mjE= zOvtYm|54md-p2bK&9`$tPANtDyGr110rFM5p*@yFAa8P_zP%M-M;hdbv%%NTM14}p zKkf$mGsqv^1h$dyX$GfOBY#~F%5R?!&ZmFo)qsCl4SD{>;4KmlsqwdH0a$%*rK)fG zLh#p8pMm@c<0l*Cg)32hg#5GI->LN3mm&W}n~*=}a&SP-%ay#I+%4l-@zyIKujc-A zsq(}3fZZoTf98GQ ztzMMha6kBn3E-Nw;OBXMWm*T`o9AKqxA9b%le&k-tzS@N4WAF;OKUVFr=yUK<7PLnj`FDvol|0f7x#=3nQ@#M7dngxSzaah6Jk&3#5B#pgi^~3H@{ks=>sQF{cmU;R{|3%^3i1}Rll#TC-ysjP z{g&-28*}-(o$p5^?g^g$CiD$T1nc%+y*K#u=b*ofeEHMhf+WbVVEkV*4D4Et{@X|% z$^Bc-evr?Yg7RwA2bap%5VmjN0Pu3XeIA80`i<= zAU~`OTyQLSK@IfPIl+62y-NQUazxf+#le#iv)v?;PpP_KY;gz zZREZ7guc!I`0yFew%!EBDoAsisfKc{GLBe_`JXNWwh2I((R_Zh;)7lXBZ z_2iS)eT&G~leK+=Ymr{tS4`ISwOkr&Py51H{e5JuKm9Vuwf+cM*RP$d>z8pQ((C%! z7K3kA_eo-J5&48m!HdWf)P0i3o5|Y#O=N9l6CtHx(ekx)qRlYPa_{P zADm4dtL}e9UQAY>%cr=7e9KC3!V>7a_ExZoT&C`SBz-=4>=JO4JY3!Xhcib|S07pXNA?ZSul-{QS^LLkvbHbt##noD$$Grh zko9<}Cu{rGlWk>apUvc-*gvkDpkMn#z2>WszLBi^b3Ix2UmN*a_TN^r?!VLq=r5!H z+w?dz`H%!+4)J{Ndb}sxhVt}yPa|vp$R=z5$Rq3VTS(U9H%gw$`C~b`^;tU1;~27tR?IDa1~kiPa9c}uRgNwkHL4KeC+KkLZaKi8A>{NF*Ae{34__dlUe*MAv# ze|7&a^KtqL$hE%}k+pv%Wk!>sLqC^;<^P<6|{h`*%B8+tW+d<2`K^?AP_nAx~28dq{iMko9=4 zi%Z``*5kjOtoy%*tm~U}C(6_QnoidKnn#|<@li;Al|I9I^KUa* zw`Uhw`$HdD`+xdfuur#-i`?HnWW632l68Gnk#&2tk+r@)veuVzH_Frc^2u6XfUN6N zP1gRmkgWZ;j;z~%8Cj3FMzSuim8{F_B5V74`_3a|-^0tt*{R#KLKJ8B#WL=*;vY!8n$a?+?leNFslXZVKk+prB z$a;S5BkTScd@t-eaFtmC8X`ypRhf&Pt>tz0h~$htjR$hv*I z$hv)!*FvB6hX7gEw~?&(7hA~M9}*uxe(gW0WZnM-WL9MFd$f@Cc-l< z&o=?G_Kyfz_fL(^&;95kvW`C%J`DXj{%9ua{@X&<{nz>k((Cmm=TWd8Z_CNL|2oOK z{{}T9z4pgUvhJU3vhJU1vTpw#vTpC}$DmL9YY|zuf0(S>znZMue<4}7e;rx3e=}LP ze>YjT|5mbY|Fp+pPycuz>+xDd*6VkatlP7mtmmiAWL^J0vi85!Cs3Ylj|{S|uZ^th zS3uVGM#;MWmyz}Uu$8Rmzb$0lzp3j{zV6>lvL25`WZfQNvL3(Hap{+n_4sWeYk%n` zYkwK^B+A$R;v(zz$sue12#~dZtS0OBY9nj^=ppO&OMD9DX@AHhYkw#t>-McC>-sO! z`8ggN$l4!PllAyadm82G{t1wE{JfB?$LlJx_NOMYjz>Dl`h20V1^V^)ttD%JTSnIH z)lSy+?;-2?JmVSm2hTH#$l4#)lXd^Kk@ft$iLB?}Ub40)@AOUT;a*O0Zpw~@8Kca!yeIQRwV)BT@8*8ZPE*8ZPQ z*8MY`to?ZtS^KZ64f^$ZvyrU*Gx-eFWtj{z0$jf=YpY#&)>-oz>*7H$5 zS+{S5tlM`@T>5Tue}BIW{kp%i$-2J-WZl2DWZl2Z$-4YrvMxV)BlPS3wUKpy7Lm38 zFC**urG>29JNXsp)BEc@viA2VS+`FuS^NJoviAS=WZfQJWZfQvUPXDjJ<`eA|MSSY zeHM{*{p-ovpBu@#{u{~KKfB4gJ(B)~^7VY+BJ1`kAZ!1uA?x{}fvo+fnXKEVi>%vc zD_OUXsU79(_8CRi<83-wkGExHJ%4wRwg2b627TK9tI7I&a1B|v?hVU`7+k0iTo1nZzq4w@xFz; zn*NjUF7z+q_;!&OP@Ye&ro4!3XL;4+S6Kf#@`U%~s8PVywS z*WmYHUpennP2`W*{)Obj*Pwl>$*)jePd=UPyPEt4+oP2nrGIpgZ)JP;lEb`jPTdT9 z%$zSW$#3w!D4RTt^(!Rb&-=P!@<{qeHCg>Td#ZmM$Sc_YjpQ}-m(ApdSf4HAN9Yd; z@58?LSf6C_V{DHC@=^Nyi#((e<#&-2?*b=&0DaGL{x*>hazm0qw z=bvu!)0|%heF*)Z@P0m%{4DS1v&r`@f&OB$neDlR{5$7|M)GXlhjx&Y79;=QPL#JR z=ZAFi&-C|EwLjz_A=I^L>{<3_TMM_b7{{@O&=@l_XD?_Yb#I(|z06!z)((?!Pm9TW zaJ{T1>+{w{WPSfwuk&;KuOaL6loqm%cRI*ge-Bx&*U6v3-l@0C`&qjsB>eOR_|{qA z=$GKFGQTK!;4AQi2Jp+|R;E9x2lD%+|5W5_*U@0*EDnaO?}Uj)xX23>^uHP z$akSVw~2KFY}G!m&hqHpD1=F43U=_6B0(S|8F60 zlKD=_5Bvw@--^F0o=09L^R43XyFh-q#7~N!A@3{mqvE4?g?u^v_XcvNtRG7LBl%gG z&lR6P2~|HmGXFQvbikZ+;B+`*7v zBJrV0{|Na@nZFfVhCqHY`)`*$!4|fE1^Gz!&ky9+*uG_ZA^jP$o+*8wlYe9To{;k2W{&#C?q&LICw&dU_n9s~J1 z%KwE&91A{K`J=FlJVDJD!o}p9t_LTXkiJ&6kI45YpR4*`_yF=YHU5-7@>R0FDK4`j z|4Xu-E3PAtll4n+3wgNm2jLzP=v(!O7rbRey^9=qccl4}jmw z27jXbN#wtgU!y%IjE4MM<-a07jhv?DN8yw)kpDvch2%!sn>`ltSCv0X`sw5nHGc|6 z$VK_`{;F_C4$^-^e|d+Tpyor7*N%gHo5Y8Te3f)5ug14Q~ zWB+X>AIbhsJq_i#CH`0GJ>;*MeV(LLGt%#N?v;jc(J@MuXqvpp)!nFGb|Z zZ2xL}p$nss}HWjanJePdRgWw|a8Eo%b z^5c}ZlZUC#CzAZdb5UNBZZGmzYW)y-Gx-&@9td}nccuPba=wa}M4mqn`XbEVPOjwm z+d?i>@wX~}KGIL6zb4dxQ`Pz?@(l9lDjpGbk(a6UPL)TVt@cMsANf+Xeh62S4^i=h za0_{#i@+V^1KHnO$ZyfV6E1{3pK-jq$nCT@?IOsZQ}L$gH<1^!e{;zva{j9&zeIZ* z$oVQ>ko=A0P8I(Mw~_svzdOl$s(3@>z2sk1{3D!lG3>d4^P!16m-=$a-E5B{a!{@3 zlD}G~SMjHC9runwR5%$k&@)Q+6OZhEY&i?7p%=X+ueqF_9l0IW0 z?Ay%wESr2C+oO;?k@n|ahV)x#Uk7;=$9ux%kiX0JH<54T`jkz6mGMU*`L(sMCqllJ z?bA%&!1b*C3h19W3+cPbzp=j(7D0YL`zwijC)+23d^N{MHo2DbMIqV7@e(0dtIt!A z_F6=Kg5!TR`A60#;YyUhiT0$B8!v~wx#U95|7nYnejVE@m)y(oUw9Sdo$SBHCE)Kl z9$Lr+P0-&#p0OOmHksq zp33&9BWJO`tI6NdpIXQt(EmHgLDs*Ud=ST1!ZomW9_PO#@=&gaxmwP6sfhg6{t|B| z%KY>!^53sMuSB?$y!dEo58+;Nv-*2<;iL^npL7!B8RUe?;B4||M~i<*`dV_md~Aa% zZxQ*0Jm_mDzd0VW+d{9DK!^7r{Fe=k|lvg^As;F-C9i%3 z>DS8VtL!~g>?1#@K2Jcnf&B5QQXk<)@*zXPZRC_5r0*mz9t?Red7;|hNdBZ(QQl2U zz!~J{?gD3%m%3n2A^Dl(p)aD-%jX9udlr$`-wa+(zEAc>d+#mr2Kn+`!QEsB+4V2j zqdpdVZ~2@0Lh__tz(wRl%8SXW{wjThEa8g5wPf|rR4REL`3mxK^5M!pkvEbjQhyuy ze)-#QB`<1+eWT^`j`o(%s2Kbjc&L27gyJ;vh!k)J`LlImpU7S0GWB_a!rA1Nl;@H| zhnv4P2`*)IG?OO#%LdD(T4r8a5{LE*egGYSK;eMv3DR}D&-C2uV(HN!!Jtt1LgCj zF9xzJF8`=FxpYTN`gL*omc`{iDbBw0;`D{%SbeO{KzoYhT4W$kkJDEj$HOG?!1OX* z$MAk}_Wcl7A9I|1ij1{^`c984uOY7fd&Sv%l*}~)^WPDdJ~NIdh(8QWKRK>FD&oq& zI!=C1oc!gu`pL&R#^}F3&c40k>M#G?Kur3farVxSD}SV{_XF)eHcsEAap{kUlMjol z|1NRuxiqf)UEp*)K z#_`v2{&jF1H^x+ewF7p zc~rkADBI=PMV?*d86?kc^6W0p9`X#9XNWv|%Cna|iSi7UXK#5_y_4kGSDs<=>?cpM zJp0RYfIJ7vqx$9`c@CE65P4GNIaHqE@{EutO`gN#Ib5D2lm&VTDKFAHo>d#ZP~*Hd0G z!{d#3{l!5~#9LBYUgimuRL=C4m3eaH`{=T=N#1D5?4YNlED)^D(H6`IMx!OA!SYbR ztL!PCli-nG1Z zg3%VYC%ww)OE>-}?NS-_mPG{u>NkOOi))}bTwYmK5!^1do;f@2qX0s)ID4ptUj@uvX5kG+SsyxZLjz+hhP?bw|8qC4N@96il8vO)BX0 zS$vj&-5m^=0wJ#@UY|W$KC`6USs@N5e(jXb5ub@V$4;J9Fs|@#sZcQJw}t{Cm(3n9 z`z#jkPAX(6DYr}im6V%plg}u~Kcm1ic65Hh8B@po9Ss@?S?yk{J>-`EEM}8yC*?cf zGw#ZWcTQPxdHGDUd0PJHvnF|RCr>_Y+TUVCASj*b_4{3ZThJG>|7Rnd{;GQWCICRD&;Qw}}Sk;CRr&W-l!}c6=o~D$uq5Bbiv=^ zukL`&?eP2D7Jty;GKEY7ZHZZ1)PN1l@v6yO8|0}BmU<(_GCg}Bh{=Fo$r@W3lsu)8 zit=dCFF%E3K9+Bu(}NW=f`JL)a-TO`ncx{a7R@~&SQf0179TS=8kArE_p8$AK;SG{ z9wr7Px5pE#s3@;U@RUb_Wq}g;!7mej(D=zSb<*syl^58?p5q9GtE)rfJWffk0tLUt z!Jjx{o$autSYj7QB{RV1Nvu*whB+#&Lov3^MWxF>3}kCDSoG7o>IwWtk}Nt z@~GJ86)vqRlfIPi#{Yfuf)(YGXnXW9F@IuNNd4*$TFs_y4~a{Z%qa5&tNp=9RHBaU zs&ay1iD1UbhQ39GxDF zR+RV)3XZi5mZ$fQbI{EMu1FQTB2^+0B%Bs*K&u|5C{@RV865s8W{IfM;$=P=QR(p+70y zt*1CF3~HJT{z(>N1_^|@l;|8X5y=?;LcaB9O);R@g+nJtM zkY12JHhpyZgo1RltwPPda=K6vROv-q!0xcw9oCS=XZM-=wb@j>X^liHoED=6WPnA& zGA@-iW8;>U#hqJLme{SfZUcTZPJU4_6+62LY9nlv?JRbg19n@$W;3~LX1iZoE=H+H zOO8oOrB%a0%WRV)-&-MenoT95X-2TZO0czos6!GL79Icz4k#c2LkY>_EuJ^B3)jS#b$Dvy|!S`p|jgeMl(oIXH?T$xkFXRtSlL7%7y8se!tUcwmZxraaNy0t9Clro`I|^HD0r_BIPA= zf|!++Z+tfl#Ap*|`ipIDahhB{t21bqIykKTMai~Goe0`zNbeWh)z+fgViD(^IeOA0 zqXkttIr)aYmU6#8Tonj<)ai>fjKkruSRFxc$Yk+4Em~#&fHq6}syS`oh&JZhtgP8V zsnnT4zfw*Mva$>lq{c=7Dc}9FBl8z$Wg()R22?q!tQKXDe#8=|BWU&7{9cR2Y;(#< z47w=x;K$wH3ua5H42}|>#GO@gXtTr%5uqTb4xqSothn*1SKz-)H=odKuoK~+xcK?57m5s+Dv`ZUgQ-_eMwF4HXLq@MJV7r(~HMy>qg%b~-2T=qcrq%B(EaNit!{|5c0H6iRU}xBO=^h5SBe!0Xh- z*^Lf#N*NyML$g_ChY}f`Gh{W8JQl0n5eT@wA*j<+Ht9;m)4yJCdu2*q3GG2ACBYn(yTlrIfP zi8OM(l9iQL5{-s~NH)+^Y2`rB?6!OD4!0c9I=wMw#&~D3w;~v@M@nXwN6ltAf0E*@ zvi|x-g*RZgI2=JsAP|s2tyN&2L1-oGRAq5_)ILoe;F(Rbh2`mwG9VPP+SJ-*cG@g% zo!e~V45&tw9B#V=h`G%4Cz>ZE?AR!I0A;PP)@6%%)~b$+Zpr)Ek379PD*L33u|%wA|=O! z66DG8;%wPf%CbJ%?+;eW8Q;XR*>c_xm>Ae`u0>cF7YR9i4rw5}!{@b|oYJ8f`3?kq z5<+_ePIa6jw{&LfX#8+`u+$ff%=P{y|Sri{C3vZM0G#B4iDZxpLeQPq`|>K9ocRg4_+xtu<;%^e5@tP&crV7FX! zZJ%!juJcBx4s4czu|TElZe;&nUg{AKl=f7+o$-Pbh2Rlm(P^q4{MHlR+U_HsYPh}PpS>A>UzxUwaJOJ*P(S_tuuzT*&(f6pxKtEF7x`G-yNxvWmRrdW}0y~C&%(~cN(mY z$kbP&f-ecY0@CdkhfHX0lP+`nu-I|V<#b;aEs-@tT{p=8#fhhu&6pOHGjSY&%T@(z zlFMbY_<|uBc0q@1!C5CYba|DeZ*k0KW0J#_j`9n21?CjLkcrY|*BLQ4srAZME_0jN zIa&U%S1rReToR^BtY}ouF3ssQ2gLi`Zm-@1#h`P2jOCRvYFw|PMV4PIH%3! zmiR?B6E=sbznIwWH=;QE#4@>O&;eb*YYT~O5`vh5PKUOxzXxqQ?uR*IR9sRvLk5ZQ zo&Du7TP?D(_#95V(_+=y99E;Qs>wYF7G$%Um4&VH4)yzkaL{R&n^oCtT4g#823+hB z3^Stzv2C*Z)ghxay>xw0Lf=rx=?VpXf1X|fN3+}OcSv;Qx4U%uzrCxxE^ELUkpKBZ z0h5?U$1)nWKdN<0^vumkfj-Z0sSuaDP-z6e1jc4x$l`U|wGtn2aTf@I-AWFaGT`)h(OS4^IKw!+u5rG@-~E3ydLir2zb0|_4kCzB&<^B zy8)bgiQCvEPzYG$7Tp)JsRN3g4gCMKku%1XI?s{aoE~E~n^!_5f5;jPNaW?*sXilq zQAb&F8e=$u*kO|RKHxCf%r>9J>irM-jE$LNta`yFpo%Rp-sr)+B(5TA43Fnst`g7P0FVk@HdYR+N+^ z+TF4gS;Z5B&i-uu?I9;KI}GK4u@R>tj@q!?3d$%~Lw(@jR=yCkz{>`BrVK1~RHJr9 z3i9OV0qG=;Q%Aj7St@?d$|{imE2&;&jDshJJ%2D`#7ETIH@3=+WcrNHi_$Rj~hR8qFLV^kJEd}BCH+3pIKsz%5PQJ^gQ`DI%qg;mPU5SQ zsatI;jkD#+hUbhiey7Uo-&i{2^g_;(lz~3C-Da^{Lw2vp?bZgyo_<@-FWXl?m?T97dl3B#U%ExXOoIZ6`ET_`OMTl(o4OSfx zFv~RN4Z4C3*>LVW^Ir*&IPkXL_4#F$msryka{KHKuU4!E9%^l|=@rpfAoS}=>P6J}?>0)%5` z+53j&0771|skHkfk`7wsg&~W5Oo9A(x;LWQ{xn%>&eXxWU0yD%m@9T!W$EzCRh7kR z7R_3%%Wk;YfN(kNoj2EbnNXHW`BC=b#!{%F|M6->?Kpk^!B#0CueX^@a(NiAJA-Cj z=wG`v#Y@J1_13l=XUcY1HLlztxC0iO#Whf0|6*XLGPe%@UFT%`1_Q zRgMp3V#4s_1_sxU{ri`{zAuhjapl^~Di@qKQ^?_vRV8MsP}gQV&c;TE^w01Z^u}aA zzCtE=&TJNw&nuoU(Yd_wD>r8YXCUbmS4fsMVBG(H_H5$< zBCE2vq;&gB*wQX?$!GIBd{&dsFYlwq>Q>>pX~$cL_;ptwN~k-#aye(2rrx(zQ;+fD zEw6U6vbNizUdK@I(625hcWNuJRUKN+Kq*=zmV^0Sf%IXr>9SqXRMA3W3LbbbIXje zf&Ah+auiyoZoPfHiYPU)$}KhE}gz&Yc4C!mk{+I<%SQ9q;I!`EuSNL6lo| z40!v|=qov^lNraNDl1L~KZu=jjGdi*j#oB>a(Wz=vw3-ERDneQcDdYg$m^$WnSrf3 zdw(0bcHEeuiaTDd#CyPi$SXD$mQq#x5b`=q4wKn$?=LlG17S5AlT-gY1$sE*>_En^ zTY{y^Xob88FA7Z#yUS_|inj)2dl?G;Su0r18hzThsh%?nMvuuGC;43xZJDfQiv(P9 zyQTB@H-iKPJ4BlQ$yNT-@8efyCjAF1$1xrpws`~QU-x}8p+ zRc>TtQ{a+=rK?UGwRmqg_*mt8CRe}1+u8)UUi$n@=$Ew47i@QwjDX09sn zM|Gp?l2?!;>TtdP2j!u1H}5vf{BM^TTh6pNU*pZ0KWvj7&sim|mpIyI4myKYt5)TJ zDz4^opDjO?lxJnhIYPC#SyZB^nC+L5P$lT!eAz=`Ic-;?L5>7{ev?^NZP|JRX=}{z zG@JTg$O%afryO{Con~9WYLS3g=fL=pcv!tCWJk2oKcF#AL@2RF`J7-yuH1R4Kvusc zR1)`=kg?U2zl`uph-LTbB3y8)KNvUaLP6F8HH+_X^#U6>!1_m*qq4lJLY>VQTikLH z=8_{uuUyB=ER9Z!Z3s(Qxp6^(?U*c{a((Zz$qO>-&d{#&U_!R3O9!*1Qg*ii9!gkj z(?I%EB2NalVJHvp9bWlE5IIG5dgV$v6xgW^W_{R#cNQ#8XGm@>>}GjuK@KdLWkBSm z!bX>Rttc*PoF-)((fjuIgp}LL+axB5zWj1zOkF#Mj{4P{iaF~3LC)spY>ysgw%##v zWEd^?-8f-VtD!z^lBSoPft=pU`&m*C+03%szh~zWk`>n`!M$6iby+cV`wR$&)$0#g za?f8Hk;Sr1rse@#EI1AB=cLn1qUzm|@+ukl9$DPgg<_6Da``S7p|W0k1#wO)qb;hs*drdfHKHtmp=!T<93(H>ho)bm_Q}UF}sZyKe*deuQ7UzxA^1@9t>l5Fjl(KS}g!zBfhRqUcjHUUZJE?0~s zN|+i6km7mz|1&ptPmduJ35Zmm?!SALI3YYDJlxG~*tU1hu5P+!?OV*xaY}h{ku<*pyP*bww zOIS$B3)EV)v)zeeL^j6yQkGPc?(Fawl_j8&qyVB*^x;5~XiaaMOWDeQW|ASQ&iRi( zG}3E>cc2Q=4Yb^HL~@L{Yjx8SCzs{IaL&KC&za4mCrJmNoajIAK4_P6VgTF~#BRL2 z;a`E^4n%;GI0k`3#?rk@?Q3p6p%9m^ZE8UlurXq>lo_{F)<2W1qob9j2PP6nt*R+w zHmM)Z<*(_k-f;n>pSmv2489tYfW~Ak4)=*OepJvNPL7+@E~xzF%rIiMQIMUF` zg2`x~Z9CjqQa^aJ?#{=&+5!?cKX#F3DLbX$3{yf$SzHC%_a; zWm^S%XR_qu%-T2f#fQ#+fyOg`e!|1+bN9K#;8x2yLz!lfvVut;!Op zR0fzfao6+v?PqaGBbQn=4)UhC5`qJ6OZuMY?10H9WHHO|wx(lZSw}U&jat=p$MoaS z^d1P{vpc(;R)N(kYb*>oFkW?j%OiH0FDsr&U0hJ_)+!eS1#vB(E(iYahCbxJ#Ol77 zga^}+5@X}xk7)~|fgivJTXD%N%Qus1Enkncn_LtVjkH)f<{dCl@X_apN3P=Mi#|#* zt!+w=avOy|Z*fmKe!eUU-2CS9U&A*aU}ZuO!3Y$gz{y~uLE2Y|Ao_W&&}vfV=WkbbRE^Umvcn3m|rFM>j=E zkfpPe?>tC)SJ>xwg{)M00QuuaV&Z;tGmc=*U;1+Y3%{7;>~~hZPsyhA4ZT)%l?=Jtbd#X_x#djYX%xi=saJ?YIp2%a6p!}? zeLBllac!>n<*XlR{$q1Ac38F#W}zli(ARC6;|ElYd(x?4H2#C3mx=55RBn3lA-1yX z&SWG4JnA*WMhd#FjOn2AX@}vbUPDd+JhGjKIM#gr(T^M?T>LdVoH_lTHiICK^wUG$ z{zQ_NU*Q>YpWEPZZPwwiKbfrOCOL1v&@6XyA_#1AAMWKJ#(tW;bN-iKfDA5iwtazv zr821^-0!SZm__xEYmzCXV%7@^nLx?qKp~+)R|ko)IBrQg>Vc9f9$_VG?<-wQdeACJ z(JcYaP?4S}3qOo{xcc8Xho6D~778rw#^&baGRBOkyUJtE8h3N=*Hjh@u#F>=Ryh|) zMd7g&B{>~JjZrc>ZJF6eFBVv&7k8h4TF!EMI24}rh0axhY0#M0Tt1j`D@QS=Q7Rh( z=WB?OvOej$+}*?l4F(}1(PAvWlU`Lz%_Apu+EN3rl4cw$f1$qHnrlOnq4{)kU43}} zdw%|&WMTQa1t6D38ULlsMX@V5?ANLg<;Cr__}*Lz^o2xf4uX^9JU)?Ld2CVz-mJ6) z4MiZ&josDNWMUtrD?)0$6oN+6pm}-u`daqP9^AlauW-MQeW_QnQXIwM+^0d~b2BK}WV1OBOfx*=UQ-YtrzEQTRZ3AlRP z*e2;J_wIg_dYSFD4;?~r=jjT-UT{{Bt931Mznf-KrLx$1lT5ui8)}gGT_1rSE8X2? z11*`k@>9tt805e|!Pjh4LMmQ>e&vdbt1^pWJ99%2#|NlG&CY8bho>@)GO{kkk{?`` zdLib(r1j-blD#JrV}ED9hh&muL#FuNU76Ahj@RrS#2EwNaLR0`aS|CD!j^!x&9=(g z;H)FxMhj+2gDu7*SnI3Vvbu;OpfF@>Bf*K=pFhM7R`Ef4LN)l#l=Oy9LX*P+j9bUp z>5Bita^Qw4b$k2cV*)4RWZK)5ydRM!wJ*MfEmbKjS#v^VrenIm0S94t1&d}} zPailVJfyj-CkxlqakyHV4&i9qKSZ3MO)xPyMdUCx{M=Nil;qynZ|Kwmb| z4iq4tH+PV?G3UVZt15+c5@#3w5j()z+S8<3n*tw|vqhrteeJ?2##9u+`$GKbl$#@E?m`84xoMftxl|U|K8bPp3`0cmWMl!*(hojF-@`nUMg@+{L<`6! zlK>+!DMe!gD$v;`A?peU@|t+vl-F5WxFgMENCOB+mrO6t(X>$}br0bX(U3F@nwL?i zq>n8;$1YFYq*+@hCg0sahvYlTdCRlrNi4PIwo221J~{jPwp)G924ZsIKvUL;n!>fD zXZrzYGf^_cK~M;!@CVRiF}YMKm2IE>TX`|X*)Q>H+had&P87@whnmZ>Nk zshZk{P&eaf1PSyBU@`GJsY%f0HW}m?AO!2AqX5@YU8eKaUq7iul{PxC3^Q>$2@c{^ z(j3-THbg_Tj<+pm8I^#9T9bXIU`bp06hlx^ZzX>qfo{S|m$Qvibz_Q7;F?FFj2KM} zOly8Lq_7M0SvwAPP-m*;BW=qWjg=`P?GSR4&m|OWQ~n;1Aq4)N8L4x4BB0Itrc=Y2fQ6h0*y&di)%uBEF?*i6LDAXX0EPs-iXFuk}oQULh?)8JicLiQVJ zpaGWEf~Fy&>+r_=@EOC1GDv||V@B67Zy48y-V*u6SeKO<~66Fl^14Q)8v{ZY+8B9hqXH0HRQjfo(ixIF;OBh3D0R)uZ zeY%58Gr9IF77y3AB)DW|ux8Lb(1`^SZ|EMqQAyOtkpY@?fLzf1{B-re7vz1sINJGd zmTUxng&Q~Nz?nm%CjT80H;^n_5*e;Na&xz>x8JVX8zH|f&dl#JJK=PAmO}y%A}1r} z2Y5yxNFNSph5`;E!cX{5bnfVqF$QJ6QFom&&f}xQhtotTJRDRYbVlaa-P5CKddqkgaDg^eayYSI?S941= zFK#~@j6YM2Y6FLP(ZYd7TGox^qX9sfi)dh;-ue!8y^|C5dgRQR4?nL>oe?AnZh|jv zLXfazcCf%?o$U_NkeW?1($xw#h5JlTM+f#bU5;>Aft@Azb_3O(o<(3GE7Ul`PI_^# zOxApWj_$XN69&*4(AUO3j#vO=*#9!d41!IVog&a{G*~rm{Q5^nsHLixNV!Qe<@l^A z2Xx(#mzeFcdRbC-8&|yGkoeB{vY}ZdhwJPjl#@zBM>oJNR!F|K`{DqG1Xz-dw7-__ z0i18}{k`V$Vr!Wus5X#zH%@a0@mhtA&O!V|g?k7+n@mSVF2Vm$n@RyC2e_JO1q5Lv zhF+#|XoQ6c(}>UAXKnyz6u%q)QJj!TBO%mKgtFGdCAjMEGD@*s&tQ0Jnxf zXWXR5xG#92&;ejHi2?PELSk;0{nL+a01WJ&xF8ci%GVDhXHI}XCp^{ z@j?XU>|}_eJ2c>=BHvZ_u{5keK%(!nw|cK62^7?Cq=+kilB>oWx0f9CH>ps$qVCw` z@jx?iBV>}Ln=E<95@<&!NPxutW@F^flrx&gbxq?Tp&`uycE89HkYnNq?6=FgP{$eW^aRj! zHBAeke}Giy5gv&fIzF=3e}qGgj2uvIDi;3gl^{Kn_I}nM-~>`NF&%H5^c?m2CI@au zCx_{j+YYym-JB9SNd)qh5J5y;?#7ttWaYybs#5tY`LFQynvxIqCn4XLY@=b6z|LLk zD%q9ewNdWRG)mAn!Ov-hh4DzEW)l? za@}m3HEUC*2V503%^I;3~)?eLi_@Sxf{XEld-gk1>!fCf>Zm3oC3_B zU*WV|sR#{pX0eRA25KUPwNK-|qZ&YqNT@%6(iy}kCGo>826EUqy8IEh@kU&`2DT$Q z?9z(lGHfn(IP++ec{y;744~)|&Jn6FRZU*+@s(W!O%$648YFlS0(i=KO)QWi(g^NB zG;t#wWcv)OEiLo8Z7G@Um`DL6j?;psc{|c{YwpWHXBV{bWpDUTP56V;0Y@DVC!B*F zVtK?ZMNTqdW`?le^R`@@DpGgVGH?q^VmCmXY)-zFPM4Y%7fD{hOOQf^V{h{}5GS`_ zP~djq{H=-zmLPqPfOk8n+_fFdd1TGAh$ zTr$E$I0>Q&11)Iw+Q6j5)ql`9(K>3DbO=>Tm=G*4watTl4 zE)(t^I$Yc}azjDpeD~x*ZXyhxwT_~cY9zos^d0?uEkDLo`Gyi+1h}u~+&X%zBQOeD zgwGEK7rQ?B2Dk^T-Mmf>1`)wNYkhTX`;*IsrfZ54-PECv?;F2Zj>LL|B7pcTW>~yz&BsFh7X<3 zD!^&TX=wO`g`XqU+6zVL%@62#>how~`h{hu5sLmXf@(MwLEhTi9*n>;;ZaMXGfq2< zhBz>iWPfSCZy_icieHk+bQsLC$b^uYb~)1Elo9mJP{S|<1mpFa9FCL-fEgS0HBIWM zqsan#ZCc>Y40Pf{gVTnS{J*6_X{%}&*Da^=ASykh=QLy#9jbP25+>Rbh+Op$$| zsb67E3P~K+ zvYgr^Afkw(ZfMP&G@Fy>8;25>d`gdY#XW}@2_gXaBZ`We0_hSr)DzEU)P;oUIw$7> z5u_Vr=QpR3ero48lk9`^>9GHdb`Yc`r1>bAfg0_0i+W$JQZa$~6(#bU;=R#6E4QS? zN$bm+q$U8>Qd1-dx%lyx4z!l4mVMYzn<5`JL{!g+bKUdIj~FS+e)A%6+_p(;;EsYZ zC43zR`x;N2-}PtQpa2D|Ln#$_nH%Ei357l) z1|-seg6=j*#cY-UKX^NtGo^@=C+Xtxanm@q68EOrg*@x6Q3 z=j_4~R4qao9WYBuAVX~k!FeDGe@r^4l)A5KGI%6OfU@aYoE}+Uq{}alKFFK9rr7<)>R`XMGrU*<`{Sjg73En6hy>ZQ@9w&q}3Fy?G@9l@!bC7 zu>lC4k!R{zDHwKr8XX&{IlS}xxWO0UDAk*bGyGUdbMO|LpbguNKU3VE9NaK&vlgRGb>o^d(lZP*38Xt}DWOF=H8on|nKsoe^v3o1J3jc8dWlcEzX25p)^x8WZMc2ay^SP+JB z3O@z<33(?jfc;EQw?hzivf@aaK?yp5v)?_&0;7~0{l{d09ock_Bi2c=F<^V?M&YwA zd$8D$AG)!;saR9dSvfkN311Me1W#w7sA=w?=zX+POvfv#e>R?XzhX>BQf4^^A8TH; zOzcF|nrgSl^jbId)qu>)-C(q^ZR$b`YKsc`fqsn^K4Q>IlE*1==;vQyQbk`m-Qq~q zY+rqR^^#YiAwFqEDG&d)&)jJKJFMhLp6Z6o#biPUga?QS0wTn&(+0=GKfu(v4T=pj zW~15>c1VmvQVnk7DsA9fK)%#{7A1Tz_`n@dVH(lzWA`+I(aqgmBPp7O-d>gk(f$kh z8%(}5`?5Mx(qP;^BaB3Sh^;=N9_MaPC(8FW$=Mm! zB+>Z29VD(bqCVzK_mnZg_VRQ0B>oj~DV|JU65Le{&MJ<9DteCW`A~t%C%&_RdYH@zgh#}Q^2v*?_!E<=77cI+5w#`NL~?8H+UH|Ct$pz2 z{2duMUIg`rBBPRz;;XyUGjge@LpR_{dT=f?&9#M5bg-Y$Z$q0(1(&kB*dJn4&lYjV>M3Tj zb3xCS(;ZO-nUE^DeGH%8_E&x*#?W+CC%K^> zbkbWi+XM*ZeQjGGEboplkbpaUPH=I zUWdU2^(d04BoFO`t+O_K5(kJBK7C*07wKVwm11t=mmTG|&4qxlt*PG_r|b>b)IZMz z;cBfr#O4A#Bm;xCmLGQJO(a@9^g$0KFc@)q$-$e%{vz8;JwM!bZA{AK;#!!5&*e#{ zT$-V%F71SncXa7u29k6sY_6`sl74T3Gdhl2Y!5MU=(jl%tJ#WUD6|~ zpYBe5)k^CgcFn6F*O9V<-wJM9NbkuIQr_k$8SbARjlvb4MAC>3YdCB?;q-B@R(njUGW{r0?QzXna6okfuJPm=mx1~ zi7@L51@QH^JFyvX+sI9cyvZJzdHnS={nQ)2CHzU*@Be&r{_5<*i_=$|UtWBW-2&uEzy?&T zseQQtjVtckzUrS{U##WIdep%oDrXxZz=lK^OC@|v2|?)FUU-RiM*d&Z=bct|96W?76f0mH6XgSeLVJ@t+YQ90Uj2?UsLrX z!X4Cv^`7NA5^5h*c~7Xlz&|yDPNI)Ir-QzwCClDSZN;J7wDOhgxy`IIA)#0#X`q6$ z0tT&XD=PYfa%D;OsJI&}>+o+*qksJha-jO35L{VwhU zFFA)V+PF_)#lH`YDY`q_C_8PGlVywS`t~t=zK0)wa8(C!<*!lOmU0qs276cg%Req} zTBT5Y5dN zD`JIv*6Ot2ubz~)ztS&%g&+6o&u%cajBrV1oXV6|KxL28v7&b?)U4`fFe9CuxCDVn z*J-L?IYXaeUREph0EXKu$==bfc#-TLTy7p;2%rpbLZdiJIMf$`inIj>(i4f}W?2L}?Dc#9glz zj7b-CSFU9x4a^(9t!ShTcAHfJLmxWfo*xt5T-)qeMqKz4vt_?8@2R$x#qAi)`b_Wa{?NJU99$6@wj@>J2 z2FVPbo11zD9k7wO1L(%n#tkXRaM8z^41j_a z=;Ao}5Fob;SSf+^$j7D4FUB>4Boq=`cj1pt4UPbX_dMB+6k+pj$nH){HsVOXQC0xH&DxOYF>4LwN<=2+b8)PWQ397~ZjI zz4CYn%YM4#y<<$fBIW$rsddt$M{r0j1nQ|a_RqIOPdF`U(9EXcQxY=xz292e^=?VU z7$zwgy?JL?VUY-=<*CCSk$%<*dr~vWVROelViF>BbVFy|&z($so{N?C_m9E+;!QRKu|z}a;}9BZ>7Og0>xUcU%rCtWj8ppfZey6h9# z?5n$`Tr~F>*kv0D~K7S8T6s#0ZKZw^7xn|Z=uo!kU0CA-5 zn6#ZTYoy`vLJLZKWNagM&9o8%{a*j?rUNMS?UH8wBzfWf=){DLc|pG7-s!Kgm=JzL z*b)xn(mu%SQ_R9_8tP$FCyiTjPE%*qho3G(mxhgNsrh^K3ckX|WP&kD5VUg0*DU46 zjRC{t)NJmlT5InXIKv>~@tKt!CgEqH7JSJ(G%$>d{Nsa-lyFt#pCsls|0X;(<{!nl z(P&4S8dY!j;{B6-2Q6*$F}XoOk_Io13@9w$@JZ?Px#Ok)e$u17Vzil&z}x7bH-8K; z6XF4;fMPhb*US7XaJ(r zMuh-!hj3w%-dl;U=J~OsvD5N4G-RbwMe{8&G0bhVyRE#5SHS8+iMlLsgIfM1n$q|;i=y}QT;eG-S80ypHYrrI2= zK*c{Xy+b3D%zt6pAEhYJ^db_15wt+Q_8T~iO(RKsk^dGi0=97^BM^?LgnoQB5&LyR zr(vy@-j&l&0p7&y#--JV6|*f*Aw>h4Pat&0dx2jgcQ<@`0=|TEhpu!O?*uLG2a@=T zB@KFCS*DGD>^gBE2nTCWmJNX)v@rJO&qruzV7!IZgjo%b}3F-tfgELHd@?;HuLY>6C9GB{PxL}ZZNn7YAv z&)j5lW+wyic=9SZ2P8em++Cwa9Mx5tX0=ajn;StIBVNM^VpH9_<1hHhnRdezSu>Ezz*ZKeD3U8=XwESPV9tXHT(! z5P!i9&>^qkMwH`tqFoYa3&9fliVI?1Y6ielamwsls|zz{6*!|B{`TVP;{Mg2pPPrR zq;-R%BS=ThOQi*6yo{3wtXU3V+}>qo5G+Io1Hl_rH2`y=<${xoI{D-Z&hUCEj=SRi zKKWR;Vg#~t`5%ulP`n4cQ^3ziyWR9LCh3pK!$U-Y9Ag7xBcuw@!RVw;mu4n8Cm}{J zP4#2@hgW`qiGgDc`n8JgW_Pift7Xtw|JTDs_lNMp?|#Q)lLtV>BBmG*aX1Nx@Bs|B zWF{V2Y5gaQol7|bab+ZpSs-<1FS2cc^QG`|xMGQ34*M3s@*tDSFxtal1JG$q+#{K< zW}?oxuuAbhuVRUjRncio;rGHYRj5moyRz3ik8{d*4GM84OO?MofB*4}nJGm&zDMPN ze$bKPoX(7{!~@yK+A7vT(*#8>ouoNDB?GWoFtgzpVTYV~3Isk(LY~XuqCsc_k6ct8 zGI!bY4a^>oLF6)Ecfh-*h?Kb7ctebD$%OlzfXWGQaaK$WgsY?w5cl6EKWB2{N znGO#xfNYxz;A5<^!9yd<3@T{=;f&}Ogu=SSOf1w=MC?p zmRP{J@7!23!#VS$s=5VnPbYl}4NvW^F*^nQ#u6aY-R4H(u#7&&p4_T1AoTFFSSw++ zWXi{ji{FNBn(5@p;I1?dJ6PunzKO^blmg$3A<@CzbHnTw92t&1A8Ez$-}g=S{r$7P z)eK@w{0~?dnky8Cph>jzu*U_iPMII00g#hp{31+XrZ>c71hAh7099y%0)!%atutC^ z)mvY|9#aZ66F9Lz&eTab?koi-1-uKsLvU{hn%aB#CE@7~D{}~+4fm2OGtx>3Xp#8C zU!o#dvpmD^eKa^8)Ll)J06YXVd87JdUop&0r|LcUI+ZI4dsisE+6rE)C1Zz#LCUMJ zzui!>PzBXy{HJPv$xD3qr`Lpa(TM&a64E<7w zIwGXci}3c3&{vq~JdO=$?vttD#~E;?4{zT6<7{*K;`E=dcsaH#%~@#;-YQK|qR@Hr zpGjk&iPz9l6+!f3FwK(0NK@a1kNy?pb_gHW14GpGEzjcf6xbI5XUkvFF(jj724qYXr3Qk zsRYHco|GcYH2zKsCNisX7o@Wk4N89x((pil2E+w&U{XhUIwowXps#(#l~$N+O2tKd-6C`i>W zcp7h>*RBdu=F<-`x6EdT=yIfHQ?$|q2z`_Y-z-kN6{y5=b^Z6WY4HZ~ChUfxjA@)i z@Pmpn^0D?Utmk5^|1`9kWa!KXl(81hpKW?FEQ-y;<-Obu?WR$F&t2ivBRAYbCYNIy z-CcKHcePwtYGBt&ht5k_kC2uvDc&LQ`OQP#2mT-Rjq7i`<0@d%PNiEIc3Qg{Ka(2k zNM?#OkXszj0oKgfX9)xR$V)@p2p%-5`)4_;7(6 z?to*LH(Q7e`3vzc`Hw7T7>}{Bus*?YM=$c+6s*|Ea5+<&a%J#+YuV2BbfL~|-vKLj z%4!Lhs9mP8+YmZOka$u@PThkaoi-;*gc*q93#SvkJ>h##vV8{-6_j6*vMHznN~`f? z2$Listv8Rqb&i(%`Vnj86Fh$Z%hvs#O=XjT1G~Tqp%rz2g&L(Biv9Eo1#9r0?dSBl zsTDylNxZk=ToCIFI0LSB5bdo{5`_~ivvUNZ+KT(3XQnED_QPkSA(6?3gTrXQ%?C!V zdOqv(#pWT>9<%*QnA$9GTPK@2;3Z%KM|h3;4A@YiklMYwlik}w9|*C`nfxNHKS_-u z`5p2N*Y_Lpn!^4DKJAlrj!gE4V=ID13WOR-SvTe*1vh$aE&(Dik_F<3E1<#?JgKyf zUJZAKQCb?-$U*lAgAxq7;hm58Ty>&Zg9VIhl$>aah2|eLupZOK9838muxbO%88OTSsjXbkb39Sw+XCj^&kLd zn3gIi5pSi4;(x=8u%;R^n^5k8&4OAqJds#4KF_w374eC-3bGT5i1jF>k=5kVpcQh$ z*Na23`RaWe?FYx_=sCns4jt`J%-^ie%qdG*j zYjD@&0c<~m9LfZtDu`=K^T;QYzuBl5k7Jr{^Bf*)SYik!5;Cy*427m4FK^iVn~h7> zHn&h)YTwPx{DtRwBZRD4=Dw)L>)>lKR*=ibQ)#*ts2 zlGO#~5k>S9)Af1)A^10 z4KEbFXkhy0wc1DsJn9145h64HBtJUMl+zCb78T~HK7;K#T&uy3B_Ss;5D~^g5nG}a z0U#NsK-N3#Y6W6jDb>LdxRe{?>3AT><=HhU*#aninS*fSkg%v+CbiH+>V&xkNh-K( zGpI9aI9lDcAz;VgHXOOCK~afoaV&Z_=d`Y>3REyVJMrU(`S*BLA{61J3gr!|r(+E@ zgMF!GKqJ9wZerB@7qEtZW&&$xcJmHn+Ln|bAmI1cKZ7WSk~&;aKrKj{reM{>*~PG$ zP8a=VYpKE2PM3?y>{zbmK~X~EoWy2}DX84|)4Z>hb%B%@G{_z94K55toK`5&zx^&`Y0Z*Z4AK0lvL=>?*OXOJphBt?&3$PJRwOrohoSNF&c!)NEdaG zlhd6^OPP)qFFO&eibZ}QA{<-@Qy+9Q!1YA&6UidLP1P>*Q<{*+XZ^0@YT|Iz&>&p`6pee=FV4)t~Y)D z^E=@eb;w-gUL`1l8$Zth0Nd$vTeETuo&c`_l_FJss=w}H8J;gG>MTLSjA2l9a$`Kh z4nZ42nS4MQFNZAN4YaD6D4P_`L@f80g=iwAl~B$jYe8QG*>(3$8iUNI#_gH&yKkqO zu0H4_EKoG?>D8c$U|I&V?r$cpO2Pl|X3}@>g*QP0g&G|(&Qux{{6ncnWE^+t!B#Bw z4OEmbHLob3gsDfQUr~S9{wuD87o~v|siU=&b`Toe3wIy&{*EpwL-U2CR{^0t=$zRg z6X<8{Q{YMb+t4jj;F)crCpHIwN{ymRQ4W3PW1*^Al^J9t@Wvo7WIyaAvor6)qWLOk zX~X){&`h;{iR&sbq|5?~5xYVkH5!s|d?AV{pklQvKicV{Y8l0VvCuPuFUH-~Z1qAp zQR9^UVq!6adN%r9dCTct(LIE|7uBSb7%+E-cR5cGE0fB1%j6aL- z>Le}C@nyA_j6t-*C_l)Ab!9t3z(abrMi<^a#`I_nibF{lj~6h|{b8(&=_N|ZjW8m@ zN0ex4QY{o~ayQ}+B0+=?5JXcAP^f}U+}+F@Sz7NFr@D_dtn;r({f!1@s-=LEdKs$ajSTH{s57C7O+@5j9evA`bcEQmsn{DN3pJF;p9)7-(*d3t7w zpdI{hM^{bP(~s?LtiT=4O(AE3dl1Am33MSW@_*x)l?_$VqW*QDGK|&&QmRN;0Ji`{ zWN+!5%PAIgpM3Dxf@a9D)zCSNB=KkK3Ri!zuJi@g>SzBADK29@=nut@N-0{~=ij$K z=hg8zweE)&_iLk-CGW@1CnZepafkz5q4*p`ZzjY9C!M_PQTNc{5U>Bx6Lj zmb4eAi7w#13t(~rnxIoc6plfQ{x0disj>=B7@#%T4?CP zjXoUGS}lYMkG`>0@=4;mfu_S1IUn$Kl;i2o8$kBa31C;~)7%Ll`*4N8GDr^&(gJpn zDb_cRc7#uHv?F|iJ`jhxW&NKfgA3L+^@{*V;$>>25646fR#xY-J62OJMdvRu+9=wjV z^V78Xot#{w79q+i{UGh9$Na2os1rMJYKxN|r!uKRO>W^Xdn1_#(Tb6{2fU}go-9Z5#r6c4Qx#=T~ZJI*|*CP*_j?bp&X~&%%gJtyZ4XO3$;&4;Y4w`h6tJLkEBPNf-a(;J z3q43GwlVlfK+d5d+`rwD|6SW{)-DU{AuKU9m@Nppp+C?$nVhecfg^$ci3`Cf{p?tL z*yl5uJ+^D!CnVevNCk0W2b*So=CaV_OrdiFs|r+lh}<>$Q70ee_#_s}CJuEO~!5!9$6!95<8|%NOL;a(;Ihc*dLiST|1u^XM(K*2RTm=*qaZ}V% zG_*TucK~LB-A>Baq|s8HozEQ z#Sp^J*3e0l&X&R&wK+S}YPn27`rXYfHR%JT>T*Zp*+8oXvtE{?wnCcNjV7U;KnA6_ zCN-9P)}o`KEkA(yvq@($*RHb$q6j9;Vh&+oYJf8)u=t=CF@(3sBvBm4rW~W$;!6Go z6Dt8BY4!mEdnj+xwnxA>SW{u&WNBT0uE<2A5|Zbn%?Oy>8bL#vmXN>-?kf))=GX&C;Yp3&>_3q8GIG)v_go}@YB&o~lDEH;x5-40hR0|US}>?DbVHWY?dT@; z>BaVlrZo6X=ptmfMsA>K99v$AGF=DvgTR2|+U?&Z-fyGs!l&l4ZY< zRGom`Y8a=4IvtN0$gI1GH)ku>%=iqCH`MH03w|d*K4Ge~tAGcf;X-)t6R^7SKRaD3 z$Ah}a(j%&HqNMr_)FMhw&ip9a&QZ@L>TN>gD>ZBB@JIwv0}!H55%{(D2`N#hiP0JP z|DN=p8ipD8u1ExJGm_+`yh^8Ou&u)F?5PClRcs|8K=q&iv-~3;t%T(h1uzaTTms1c zgGX~#Fn=P%U}6GkWzWP664}r>e|Ys$IXM{so~N;dQj^*YIs110I4h~lNzc(L1(CH}RL_&1s$&dYoG@Lj z%%nL__Rr?pd2&%?!GQw5BN#p5@P)jhnBGHhYRiW8d3M~I2ZRhvW#{DNck`0PI6F$OKV+y^L08$;+pE? z?#tB?k3x=yBM;3N_b}6{{9{0Pz^l^9OM5sK_+bB#B!RZ#1W@Z&`CEo@OGbXpByeOp z%BF(`kLqLJ(<Qe%fpGu7ko>R)pleZQ1fB&YKIKrhON?uS1|M{dmJrbSPH3 zn@rbg_#GBLX?V})BS=Ilh+W^sU4o>NdyEM$zHxhPPK1EyL({*|kZMm>)_0Li)aJ-a z3PcQSd{W>KxJi9_q?93X0aKcGK8U{PO~Y&nbwiY5(1b7@#tGY-GZ{F+ZBi$pqunqL3&pqvbb zSj7lM=k%!B6?Sq0TAUw3(M^9!53qT%g%&*4kDQnCt30LtoTZb{AD5Jh{8QM!bh>yQ zeKZ#8n`^0N1wX(1S94Y++ zElJE8qU~~Sj>*F*^5xBNGR+f=o;1 zrX*fAVH^OK;TgeVLprjph7>O}@N`38s@!qA<)m<=*-_|7Y3&NHFm3iL($ZqEt!H#h z%y*9cGlC-FX{DQX1zB_ilgn=ZC`#CD<=uYiVkts{a;Jd}W^uPZ)iHer7D%(ii=Kr7 zTv-l0CdLIui`Pxub23Kgt)ll;Rip)AiEu+%4A&2vX%oncj}pULPYV%9#99PmtbLhP z3uA2-qA`&GsHKCBE60Y`qpkz@#a1rdmCvW?m?x2Us4KWqkQN5!R?-9E@8na4E3|V% z#Q8Epg*VDExN-*#ha2%RNV2kDM=t% zYooq^b1$XeC@>DDV7P>w+qE|>Y(M$LJmr86&a(!NE*R{Zs1ZTH+^x5#9YRG#8(xjl zT2lj&af5irdWAuCDgmIK1-~@6@?gwnl|?mVEN~lks35!Xei+IXT2Tp`!T(EO=Wl1* z!;@^EvWxGa+}gF65hPwWQxVWPm0Gt@)EqjIhwcs2QE6BtMUg^SDhZ#}LBOy`@z zeKu`!Laid>6mla?mQd8QyP9#+Lp8#ph!HI zE(QNVm^)3}v!}4m`Jd1=UeRTX-J9H44TdFnvSPtP5=s$t1dxLW8l#n;-+00q2F&n7jk?7MPelO34b{fzicw zAjaj{Ki_{m-@JeQ`s~&D=HsvL&Ug|yC(4#OHw2LsY23I^wB}V$ij=-8reUC=BzXND zeMJ%pCHRJT_@dhwDpXM~;|UOCY7#21SwkaG5{AnE>xL1u%4nY_&r$8i!BfJ}iw1QZOdWU<^XA~-=tdA}zMLtSnB-Ipf}U>}0TS1)!&B@jTyADIC>wXhoU*v?-}khU-5#={-YBd2xGfnoKBeLNkK4 zebv*y0}9e!>4=4v^ng$Es=cp=5+16id{)v@{w`Bgil^}R)cqigAe4gHpeUh)wU0Kv z$m78+=~JANIRw7s{s@GlMh_l@6iTABOUHJU=kbS^4dg7vz(GVOj~{E2r(0)>pdyd2 zOPkeD!!YL_LM))y6O1#9rnQjbqHpd%(Au_=;^A8UUJODcx(~Ie6Oxk+hgvtLfEj39 zrZF3*C4#UsFpc7U9g*&E*#Zm&go)BvY8n&mB1hy0)&$sUN|4!ThwAul0evTCDs*pv zoZ+^RuEmdZOW|q08-&mm&IH<0M5ulc`3OL_?>}cB_)ot1H{4E4GlfWGg)_?jB$ny+ z*VjVz)~4YNlo9)lBI4}>C_O$53|(l=QkuZO^jCQ6^F$O2d;t_7Q)J!n5PA3Mw0^IB z3C4G%Ssd{c)V-*t*v0XjV3N;*JXQus78YTc4(w23U_D4^KmGV>efeU-Qk*s<_lZ~m zrxSO$8$9tW2G&fDUTFGzTm?7Cfn-NgsP^T58QtxTN`v(q5#SRf9PQ1}FTm%B*(D;f zZ>LZWna1?amfdNTl(viqpy3hQ*lLnlq0x*nu&>Ba&(vNfCwO5)4OAZ$Gz~^M`M@l#e$RCMcw5!o-(P1mWMGDw8+agIENM!hJy=Xm`j_&Hob}H;%&5In zz=C3mHq3;+M!lV}BjSKt`c6f3KxHqiKYBB~1CyrYk?HOg_Q43{XcSo87`1+Pp)nBP zr8+0V{V56w#_S4Rt&>a$^IH2Z@jol!W+GrH!wQ0)3l9OoIvYQ!q5Gm{>mI^eEcVC3 zU-I_Pl_5V4TPZ{S7=VpDG}+@AKHvn@7N!dDzMb#j1Q+CY{F-UHXfKGRzSZWNZ%FHK zH6)^dxCQu+VmGWe`n8^DHV~lT=&svU83%=o67;q7m8N=|?2rqLhp@ojoFM z$_fo8T`SK@lY4ldr9f9fB#`QFg>cu7p&zZFT*_xC&de`yionGs{2?N*X*f6G@Suet z=_Dym+s5SDeaVO9Fr_E?#D`1x_U?z~a&vd_>9cG@FuxhJhNQs(+0O2B%o##i1SmXv zq@rmuJ?=UG=O;;e>Qg|ak=7%XMaIG!eQgJ{@tW{(=_(<-Go;Fj(KDuBVzosRohIGq z&}{q#NL|o1p&&*oN^dnaKuK(b$LQ6c94s8M1G!WRPd3(%|&)o{5d*~K>g8DK$if-T6U;OpjL#9<+jZ_Hs?AAHo+&& zD*|z*MAMQbA3AU|u;8q|-s90L3hnd+&>u$PwM5f8d^K^pmM64zx-=E2;2ePl++U0W zpRgw?63ZlV0AHgm39Ls)oo8|P3C#)C zssGyCp{Dy%ev!pF`N!`IgaRXkCqdR5_COp7`wp@&EZ{t+z)w?n`Br}U*I;_vObtAf zbRG+WCP>sk;U7d^lPck27Sd&QYVtnzWq%0hYpHEA#yls7CzZj3&h`1z7E5n`ZWZRa z$PgOCbkK?hb7e=nKXT9HoOKQE-TpH@l0sAbwKM^e?HPF7&%~KE&6@aV5-GY7zjG!6 zWexGiXz~ZiKy^+Z8#7P(R-nhK8)08$k2^f_q)LB#}iGFs0dII%-G8?yrw z(o0Sri>z>qrXTt_pQRS(h4gq~6ru7YlT;LEAvC7eI6NnMhr@z0A-*jXW|ZsPP$qiE z&g>;rT%+%K1aNE*0$BpU?<%UK6jz~6b2qg?&|}N#U~1c%E(LrkChFCn_s06=rfMPK z6`lboL!P50c`G7XFZ)hy;({U9xbVy?+iafW#lH>JW~T-L|&~ zL~2bwDb9Us?CYC5PNkfr-&RWAqA-LeO3D#HL$bf#1Ug5Tl?jb=Xl^??r0Cpng9J9L z{&q3I(RzI&owTAkO=k~m(salpRAE<_SrQzfCAlv$t=DmUc7LZ+ykoy1%LeG*}TPH#<;aFX)Hmvrsc?5Iu&}k37i@U6SqPx6K{wnsV^t)_`*=GuqF9m}TM6 zK`y>f&g~uH)8VRn(ngRlOP(GK6sQ1eEuZ%NVV{dwJdN<=u@rkhA3;N+q?T6W5CYCUS&!0z47H|uOS`7P#9lbA;*;&2Chv< zj3&Whf?>bbf?UVnY|Ml=_xuZ^(y^5?K{d2oLk-N>8!aVrJ;nk*r@1=%Vb^%Y4@8y1!ViPvN$|( zV~JgL|6^KE_(>c35olM6cTYF%49vtks9XioQYTpMtc(~ZBjP5z0HAyFcEJbt#jM?j z-u;fJfPHHaD%l3o#U3LzpCkN_JyrBc>7KmdF%3lu^7L-@ZeqO$E0g_%d9@IRU}mlF zPKPWdG#XnoNj6R1>o8xy083;7Dl+ zIwv<~r8~f%;ARQK4|9DjIXP*87bjWt&Y6=%KZ>qY4a67dH~g-YvIwOaD)Obf;WV2E zyHfM@w_zuHX;6T%VYA&dS9nnfuZyJ?33cdbVUp%7f}&q%;n`eBq-&d;~?ewzN@q_eKOZ_>vKA-Vgb z5Kqa>FOmS^%+FFF6*ShA3LB%A%jq8OCP`Wt2}ANm}+BDNLt!Ved@o;0wxhN%7ez~}bv-iKAfB5y>9gPY2iybyzIxip< z;BTx1*I^o{^tTsp0Uvppr#4^9fO+0@;eT-FoCuIis0w5CxYcN zq`%nMf%Dk(_n=`JeuX_qzKXJD8)p%yO^B-96;l&p%J|e4?x6F4=IiU4X7Q;W1u9C*>5KflnFzUR;1JhCi)dGk)jPvAWw4K)A(r3zrj*K8Jk z@Q)|gpeTT&2)ckQSV4o!LQV#{Kxa!vc%-w86dgrJ=!pgiRNV7zlT=pvjaupoTj zjPPA;zQ9RE#T5>;g1#Cco~gfX5(Pcl=x4vGXkk$`qCXtj=Ey(7(;`BOqU0q2CW2=4VG~-Cb6=*jNUY*vBxNpk&G%m-L(u49r zHUtqcKN+7BlBbsL1%e~Gf&G=!BXn2wD3pdH&UZ8%+E;aZbIWo9(cTp#OQ41)G2|ZO z2a)WzpfTxzfa@2%kiNC^Mt)%X0PqW+&-~6eJMwsx%{q};6p}JxIeSH^4o=8f{QJ_< zn2%>B5*ivB6xl^WrzVmH=5JqLX*w@6E$jX#xGSGjQcvKgP{5+QaA!4THG(5G0^CYk zY3a-}VcUfM95Z+$r74KjU?PeouS)yA8{%2plyo9EW~hph5ZMi-C*E_5$azpiwp|&h zw?rjGm~iExjN_hI^O2@gHWR!jr=wvL2bM`khx`qtF9Uajza1zFIHW3(%8;G3$s5O! z)kx4RHfm4<#d+`$oTp$rpLLYOjZSr6*hb?xZrvag)&R0`bP8+ONGWkUz<{U*e{nkL zhMz?)32=%Zl!VMTW8itvc_soei zL={eq7o9t^ogrB;Bf>oA&5L8ZWiGDAn;6kJ#0M;Mv-ulQx5&WtB3cl~&{|b?V0*UP zu#_a4P!B^eNE7QokEJSKT!jtjWNyt|;@8nKjF@9h?=bud&_os} z$3ANjLYarMBQ6RJbzSz%L15MwL~cqw2BvmDi1CbFx2y?`Nc01~s^39gy?OupJAiEG zFV6mTwt4eXRw7kWQh`7}+d2kirtdS4OVVa5{aHH|8H54o*kBhX8Tt|KvVDh2JWcIf z^p`{ShW{ibj@>m-Gim(Nwqpt9@nKN!;Ndb942j&~GpWYH-({NDvkf8%&fBDK{^bYb zO955sHv`np8)^K%WgTq@xfSBiBcvVa6ABWL{*6!r+xhwX9wGbX)%gpa0|j1U zVB%J+u7Hceb+=;G3r#qHT{S+p_c3kqb&_zL^a}m0jZBX|jpOuwPT#S1JDEiQB*vs) zOv}?BUzpBZ6sAaNV-bg%3b~z5vw4*4m4(?SclU7tarzmE(0>g0XF{i0OSG{OkSODb zQ%8~wRN9ymCE9U*V2mdTy#e(|f?B$=3VXw6G`%1m0Ky0HWr}T3wAz7^NTDkd+gP*c z#eDac*O#Ug5R|2*z}^n`uP%*{wG6u2;v>8jl(_V#4zf7taB?<>20Tn7E)uw?U2*19 zgc|5DBaIz?YG)^R{_k991qnM7*~XDrrHv_9%6E?e$)serg%A^!k-%(9#$s>vddVW^ z8{;0A102Y~=F*@`X%5OVd(&y35-}AsV9-Sn{aEz!%Qt5qUYwu)lLv>8x{WcYlqsm- z`!Q0DMeT+pdN?_mCq%*t4}Z?M!)yscEcKROnctg_4RlLlWE=8dUUZzt8qvQYPa{8~ z$#|R;o&sEPE;Woa%D7Bo`K?uS?#mC=r(9LtLgWDpZ} zj*udDR4DW@CL6BM(vSBjf>}2-i{`$Lcid&q^m}kKn<9gLlUAn)`4yUX{C)g3Kq+of zD%JZe?Pt&*U`@4{l?jgRd*JdOVcvvoZxGQcdQwN;$WC0mAEcF7kX9#^94Y<7{fbEx zJm2RY-Wrt3L6{Nm`oX3?rxieUnSTOw<+(*8se^VCGEWMmaFy#93}$wT`3sG?HTw3i zrZxKiMxK8W@&pf^4^#@|K(KH19NbwK0bPT{8h@aWQoRoK#DW0X$0Vg^oth~EV{VWb zwH_(oXz|9-Dx1IHJO0@X1^}_?@IkL1B$XB=2*0Ii#Ziv@jvGV2)C$S(RHPhWD;O2B z!C3Lxwd5p}5@UwwuB2-cc5yLQKt9}yFYTOpgc!$%$k&|Ewun}XlSV$rv??H@O#X(> z6!dBmKbl`>goTS8Z0;mUe-W%THtk9}l_0iv4NLYX{$4>xmsxP9x{ZMBsfqVupNi1& z9-(nGXaX{>RVl}m!aL;65Ri;Z6E4qfMPt0w*h2m8=4)C9xexvQtU)%ZvZBdHen!o42oi zd-Znn`pw&Ob2wqC=?Y2*P~ju;*Y=XaDhP>wpm7*aZ!RxMnBb?p5G*w|e|ZM2Cd@ht z9iT~pJHr)Ry7A3g3VpUkUr^mQoYy=YX$Z7*bebg#Wx8wQ=#-+b!31o}W=mGVfTD?? zQW2vdcS0iDUF8?z~ zz{-G#1jGwL2}N=D4fKH3S%T_JV-@KY1-{$ge5Dj=`tHDb(PcpKvQI16Qtj3LrPFvx zs5_7Tz9&7Q@&_s!XMUe~l*ycsxBK*sjW~@RL6Fm}d%6o#cZ`ESgM5Bt7=@-JjlmCF z#L9Ni^x$4bL(IUzfYgL7lAa+hLqMpEh40->u%0T%bX0XBR!u^$g06#Nb+DGDgpiEe z%Bbq~7t&yeHVj95EPb(@LStc1L-CL#_#}p3*xvapf0wVhOs&azF~8N`4|?R(7F2Xm zUMz*SYozB^-*)N^O~HMY1OEGJA(X4IK>#kGfys@qsGS17klHD8Y>H>!w15f*lr%5U zH!=u)<}NPO^)ROx8~EhpW-Cq{ogqODxzT!d=piq%6L4FR%_AFyj6FR1?(J+i{&X^f zLF*RY;odQeL3x+fITZ)A4kc~BGnX(}d?ANuGwiy?KoX=ZB;c!5v&X^yv~~Wid&mo$ zc!Vv=ultPs;+2K80Jo>9AgfM5ZR&%BGim~E=60dS5@edp)XNW;;+$^73rwb|+;JYc z@+as+3M|OTQUb|pcQ-v;7Imr}o^$s*He8LWMK<=3#E)Qjdi%uQe8Lbh2X^W=UxHuf zzJ#^daC_?PoBY{oCqVy8aJwARQ0h*;@`U@WlRXj$-3 z$W2joaG&`pN?yPcm761NBLM>wv``5I@u`daI3JJ z8WKpcv^0m09Au^*gMgE~VUvRXf%m6E!$6_V5i9?R@ND9_XiEuX@ z`4mdsqx@Y|oA|rL$Qk%tFdY8wdyUVm*KVe6ZVg%rEgvvENwT4B;@?P`oLjJZ*OH$* z7Jy4Eks|Ly=-yHVabs1%sjg=b>>8Zys7-Fjg^+%o=}s=syaTyk@Rih@GGO0OWzv>q zO@ghs{QDX&;_F=_yNg@4LTOLhxp@$5vsOm<;|u<1t0{&6V-J@7z*8XQJPcWb>RS){ zu78SUrYeSme@iw)7FyA6~qBbLzFO zL_rW;nFf`=`)&DK}}yxcJ!#ygIEsu>*2eb6NxX;$3ReA);VOG<%B&16Xk6KkgBNNTpD%w>Wy?*on89 zWdjoz6}Yhwcy(FluJvcn6MckCShd(#Wg|LyZnR?!czwz)H2$k{&w!)m?K0}GDNKpd znhFldP4L8k$0=BB1NAhFqH(S)7JToG;Ywi`*3KX*jC3+U^X`W6FG`on;*k9t^Vzcd z`5(^-`yjz%IY3TULc(cZcRGazD3u#YS*`#DPk{^?ACN&bKG-Y#n-$9}jj>0*Lwh4y zq5e;#TH&v@qS!y*T($PONJYV64H*vzT3|fkT{7ve&g8s{OF8SD)_^%?z_x$mI~Z4) z9{q@IM^8-KQ?y!rw_Q=)_Rg0!$cb&611OZALUhI~8DUo%@~1S~5pJ!@a>- zk)AP3BZ>#hn~-{4SbW(JWK}R{;w+8Mkm+A9^D?Uk#QaHvbcaipxuVf2w-fUk{!hLl zojBy(=%s0+D9`QeOpkz_nViBTZ4RbT8J0p-SQs;=zXYyM5U2%5z8iD@%CJ=ZOCkYQ zNE2sd#YDlp8xlX%(|z$?iW;0=05+*CWlws~%)>!r`^u6JP#z&(SJ2wDo>Q{~aZ3tJ z7g(-PX>2mieS9_gg@t7D3FJ-g(DLrU*T>&fmu8akj?f+Wk%Thu()Gt z-^K4j_!j30N;7i`F?W1rVKr8Bj~u5*Q>$cc3b`cFCH@P9WA+hjTcyNxrh0=4^MfdI z1#@M$p2?z{91t_68Re4`Tw}$Qy&Jy1p@UOqkxU{j85{AM1%$#9$=C)+x`MkXAS`gV zPKl^X8&DVgo+NkiAMf70BG+B?MAS5RP^A=BJj3OKYf5`Cdf3FdWPsc{GYf%Of!x2z z5-LB+L>#1q1qf%I`$s9XkM_#R*R-|L8E9r6rOTx3Mb7xc4vqt`?e?qwRivH%kK~a)BgugA7C*8W9tqmmMU~QI5NKjIco;pBN)OKzL5-0j_`v7nZx~! zFvt>-3hBrfCtwBK^>+CPUdn=d{o-+;s?g@S%^}Pw55g1zRbfUeC!8J7gh0L%ER)Go z(R0Y!H54S$opYEICZ=Z{N}wKv0{V5`pg*Yl3!EgS`i_!KLRB=jZ(uuhjQMhhui3^J zXast;TdjNr_T|tK5W({a-6wcNf?RhCZ#H4eUF;>he$;aQJ~V&GLGTDNN`7RL9ePOG zpWZ**-y#Vp4@x21nneFo@)P>t4g!A$ijALM{52U$@={`8?W5UW{&9KJl9{nS2`d6F ze2RMH<0&)K>k+O9S!Iu8{a5L-A0{&JG)c#FbV05a(eg9B0hu*2`pemL!y3Yu)>#uMLU3|JW1tn6zMn0F_3`MuQ zbcTY9RV#ZVo$#`N+9b_k5=#aOs>rLTNqKIzjpRsnP-O4J$>447a;+X@J^j5h&W`!Mg4lqPB|$B!3huhunthDQ zVT8Oe8R@xe!RwBL`;t31PctDg)I%5dZ(vq6OzNgq?5Xj(1QMHc7vB}XmvRDDdn9X9f=gB3!*+3pnC3+G&>fF@pt>Yl&3cDzAv(Lo1Q z#<6iF-rVG8|Eh;Fqr$bbBS(0VGjHJ08F zvs+jM;Pa_7uX;pEWu8{t)pvEPoSB$Rp2yA*aXzimLP7_RGZkkZSR*pjJog{Ep2Kn_rOFhK zi=9Gr{ zOtzSXcOV}Na6dpXdNZJV$=$uPr(oEm7r=aWC~<}Tkj^Gl-t3o~dpQeMSu|hBBV^Jj z0(gyaI75J%;q60kpAWSi$HwvVvffKeXMJ}64PAp^U575hBHE%7Mx_LICnBoxw|GWV zG#|RC(ey(my6o8cv*Uz>?PAxvI@@bFIi2nO{KmiK@ zXapfgF9E7flgat34oB4F#fU2m3>jhxH@5orDN`92^9o@M%8xpM!HS@@PHQo1Eh&+Z zhrJipVj&)}k2noc_j4?v;fUrXguVF9V9wpuzQeE@;dYb$mN*g<=h+-l8Z1Gxq!B<* z5S8J?>>L?a^lpi1`Yl1t(249087REKf?Ye1>2lZl$ZD@XZi#K)kg=DIiV$g!0|ibV zchC(Hf<`2L4T-9_u5e62M%loIbvd+PUFG^wx(%V}7Tf{8nDu+rWSFe|i$&62x1*VM zs%%%cWa;G-U`U04^VNUJuK3vZ}K^)Br zMwvGg>9ZEHe1xdb+u5_W7W^zJ7@HITatR9-iZAxEC&tQvgn8>0)-ZT#{2=B82-da7 z{L#otSE~g8N>?ka8#FR){_k*i!!i?U<`K$u+Vhj9c8?TzRLhP*o!1|vl}Zn2o+8Kx zM}162A)MKzgG~Vq{wfpK^D0~d5erxqGD37^g_jz>O4#w{&Y`}_x?jET-@M@wawoMs z0PyVZevJOv-Ztl*pZ`DOY5ylYyR97S_um)>W*$vJ-rx@;C)Bbp3(HdspakU)q>C}tfLB8_$y|RC7*zc8j1*31*%$PfS|2(7RUrYj&{ZW`cZpp zL+@Ki%Mf3sTp?sP#q*~FhMOX4Q4=8ZN-u(Y5aYA$aV;WkATa0a!nF1f5~9~cGX_%O ze!P`%$m_9Ws>iY&ZXLV`T;gXcGk; zKu-1nrc#`AN1K3HNSG?~^pu@=_=0X>n0;up08CH2LEx*a{kE94%y=nAb<1=;df|A+ zK!_HHzd|M%rrzloG6BT@b31OfkIz6Ez~%PaReN&@9ucObi>pgu{8vPkfKuKY5WfxL z>xx`=)zPXf;goyauG<6IB%4DyW8O5HJ1kwWN!o$-KN2HiEbk z^d|lp&$_v}+k7QeDW@4-8gK)`(3iu9o!g1Dajv7yC6PR364R#%YtZId=lBKt1Gc*7 zt7*cU>q>6=m$Jb5->zS=rOp7Z1({m9Hc=1J?>kRLQ_*p3N3poXwi3Z>(eVkz6@R9V zJdn(>Zmn8Dckl4|O%0v>qI|v!a~J;T>`5ycLs5c4nU}T_cPHo82GSxC1aw3#LM(bX zk9GK8c{$uVoQ$@m><8gd<|b#ElkrAqAqrKsATyhY)>e>)kn!mJOk*Xs(43WaT_&Jt zTWCDfQ5XW?Yqx``zj;jmb3Bm$gU>?zO*qo+^^;snlGgFNgvOMoR0UxS{j`!Qo5@)QKZ4y*L8bP^yLa9gWMb->3qBaMy3)E$CLHFh-6ykU zPlqZmPk;UR@s+7CO8Y1EN+2*uWk>Fs$!eN$%K}~_ga=a9gUag@dzWzimJaM8$Em?6 zmk+np1A0D~3Fi5L7uOvz53QRn*q~U z$H?J1%cNcfYK_LV)r!?h!pWh`AQ5h0v4uo^kTgLs>pvug>Dy+>a90gk=0a-aU2N}L z=-L#%hrle`g=D#|Nkf1j0no)e%V{IOJG;0tmHgP44*8qSmcL?ERiq)hz8Og*&?f2b zv+ERAakTi3+72`d;^~*>Rt(b{w&kAQ554S>%B;x0GrbuJFnIRqej@b1Emm{mPZB{% z8DghZYdLwf(B@5nQo!YF6lAD zT;mUzA32xwzFaN!Kg-18#!&>Ry6qk^?9KUbcO|E)>zy>ew_c$+ZKW%N(-7Pt z?FNwAyN@$1>^uUe_k5mYs(|RY{zksMh*`%_Jc|YxN&-?#>JGw;)v!hA^~6#?vh%wt!9@^ONi6YB`I8w&3;DnFTHnkgFs@9 z6+`+vT-~g^c%@t(tblEIb31%`0OCWsSAOrGMC!jJqm6YAE-*`G($B{xJ^1e|GoxC~ zhtYn&M_b+P(kO^xNGzgzGQylUX0tb2ArzdiN4FGwtLs@uYfblA*0`*1yA)X|B9UBG z1Jot_kIb;%Yq|_4qC_z!MOeK^yMqS<_9=r%O9Lb*==o5vbA$MFXuV_#P#d^RAqHte zg)InJVarTUPzv6$aubG=)Iuf&)k1~1ljFt&U*aI?_2t#9R`Q`dzD?-Ht4Yc(jw@9vgfKgW`^!Yj*`$IKi zLSS^TU^Uza>23Sz4kd%u>P%xrz$XZ>jtQh5Hd}3+TKg=xsaJW~LsMOu;{nM2-(7SZ zZ;tWd_Z#@o?|#w0)>6!FF-Dx5JQ*&eZp*(nL1$~BksJ;&+ll=L%f? zh|tRAeRv-8F(VPTOh!pP077-sY?a)tmw1oh!j60|^w~o{?h%K~!rTBsfw_gU6L&pE z;#L=r_`8QCsdAHqgHlTfelT8A`D7`=hSDSW1CE)>s~gRHr;s$l>z)#XjEFW4RSr5Z zc0a#j>T{zD4*RxzX8k=b1;Qq9u~I;U^ka9mvCGCgaP9o*i?+QRzFa`z3m>fIuaU{7 zvdlXp{{{%B>)p<=M;D!a&F$wnoyaaG$-3+!t)&tZF(?kc;r;-PlcD?;?(V?y+lG~C z;R`@ZOoAt&0y~f$wwF$)LzDQFMZHC-99uyP!~HT7&QdQggIE(~^c9 zPKO$=zms?851&xU?u6>mz!v!wPI6cx+!Oo#z2aUtlQHM*R^L$koCY__dxwF0LGedi zi-sytExIW>JJZD6@vIhDa*|zfjK zBnTjS0@`i|g`wT#kmjD%9A(dW(HKg{z{1Y235C@C&tntkKQ|xSQUBron8S6HoPl&4 zDqW;IH*;WbD7xUul3e z<_F12YzRQiy(u;iODFyDpv@uvZ^IEE+wXAu!e^NeCB+M12;rrB3Pko{rf$ysa5}P) znvX&*wSe@U8d=|TWUO^IY*%Uc;#;uNqAnnqX%OWG!>J8-RUemI&1a)csn%dQEaFBn3^1w?)HDb_K|1Oru|(Dp0}zrHy_Nt5o21)k_XDeC@%PGmWCEQE zQ{~7n9I}dwNOM%_4)e$vBezG6Iih#*Q1ho4Hqrxa6(AcAf`x>0c+r16TnaxE0#G2( zOXfV%_Pk)^_m5g1l_+$4ZD^v%jrgmO80!Uk&B zAE-r>qcPyhi_^{Ou_9v8Qa^z@jbbG1+~gvQs2~#v#drBW(Q8D6(GVr?^&fDDx&+0Q zfRkmDv@hlwlYks#c#MdBLaZIp#3%>_?gn#$+W4eELY~kIBN69Sh|;~Zv>r?k#wpn01NBt7h7 zij}CK9cVkKIuMHxuV@by=8T9nE6a0Kii_8wc;4ZA6a-hYeDvm|J0Z|O&CameBGV6b zzl4~y-k{BniY-1=PRLIA!k04k0z#88x7Wsro#ew4_Wy zh>z4KYp7O(=2TDz8aujFkW0)B_=biQ2ei?!EaY&&5f-&ns8R_s;Kk9NV4iJZb0`}Y z4+QX^H%ieG6>%)Zr%2ur^o}H2qag;xuYB>s>4!YbLxHMMR6bIbG(0{540&wH9?%w$ zfYQ!OUIr28c6kUcq3~!418RvJiu6m=jN&JqQOKdZholDKV*0bc)F`#_gxI351(#vDC z&CAQIBY}d*1TaDZKWa}zYzRcXZJrK%NDV|F&6;HCJ1B6)`oOo5r|;+7g94STEBe4> zvk>g$)T59(fQ(p%q`HOOja5mC2#I?5E+YFezDt{YbC6{gFT9X|?X>dPWif(+(LzGj zq+VT9Q#$laY}Gbd3|5Ygh-!ool!81>RubJCitn8OpX3i=kH_yI(nvX)gE&;bhDYHZ z8cvFrJ6T8x(S3T)*%Y^<=q?!4VE)x`R^s_2Dp>+(KNEozoEmCf>K>r#w+bJ;)h7OA zh6PeNmvS2nbxw4bF_zuX=-cFNiWTeDj;@#d5zz->kU<|$Sl6xB9c?LTTdi9QG&z+G zn1(?i6(DB^Z#bTQtGro1HrV2D+7`C+s@IMA?V70`?U{>+9doDw=ht>bwj5+p;p2ar z~R|m8X|@PFTX4wZ8NE2;i3w}-++vZX_&%7m@Gb;w%9m1S>Y5WF#|L^ zSX=EC#7$z&y;OLV3a`*sZ7U~l&Vi60ScNFL_6*|!<;A863J$*^5J!shq?SIYWX@UD zwJNo3tCUv%3Ih(3r39G0I9$lXmyJ>vyh5@_;sFPozz#j6=7jO`TimjmXkcy(I@prw z8(9DiCY9dq1>~=U@X&!#KfwP)C{zP5v1OpW>UV~qoCkoYwcvu@$BV^K!|DP3f+A3d z#GxhP5d|(>hafj%<+PB!;svlsA_yoGSyo>;iAfneF4`I*q#Y#pn9zt#Qn3_ zI_sXr9RkuD#EGQ8i?$R+EypSid;}?}Dm4|QIHj(6A?PiI3C0gGk0A(D`7SY3ofgIt z79zer*>Bo_)-~ze8WkspBOy!2HXXGM3-t_hO^CHg4lRBR(V`yk)W zMRF>j{{V0_8-aQ4cKzF7WW;xt8IPOV<`(1d)4m3@A!Jhda#RnfwiEGqPXJLJfzxMZ z^ZyaUOfZjJVtBybQNznDZ7*^^`eZ9 zyAa{fJc;DAD=8R)Bqa#3W6+1;4?y%%~|x8N_}g^ho}R>4|| zox@1uyKo2-xFkEyNXCV1P3+;S0DY*Co{0#ea3p245fo^2{zX|dHiVNL&QOpYpSXnf z{?VI^mKLQWRu3vleu@cxJVH)MvcIFahTJU4{vgXi z$!hJ0IM!&Cy17HrhZZpE3YVbEfwV5xirXzjDCC`>W+U+hI`E(IiX&G#gqYZInWyni;#-8OnLmP%5F z3Yt{#TSVj9YAqM_N4M)j6{+r4knaj2ke4aU5nES$SRW>!ne#C5+otH0C-KbOnyvd;uMyU@^lUG%RN7a4>F?!k1pyzeOAbja&nwd!o#bf8#V*=qd0Y zQVsmC8~gu(9*rg%z-A*#fQSf6vwZf#XTQ~^BsLbT0RguiQV|c16JSAhd8g)e5$AE2 zM%!;!e=3JFg>RP71foQo7M!Hx+2oyyr5E?SiW`{)0Ywkk@kyvOjC!MG7`suc!Nn*E zN&zPlgl@3jK-@v*Y{aW8TysNx84A}NNg(KOpa!BuYEmIe5CJ|kOh`1R0-}n3j;9e* zJ<3N8w|oExXJ#mv4(dNUoT~mN_BCA2oWMOHQcvBG1w~dN4!?18OnmtL0&s4N*rKo_ z)W?d&ItBo%=Z|}J-5;kg@j?jq@MmgRnA}$Af^FcGiPiC^tj(<^cWZs9dREj=2#_f| zFCD;sFmkJu8xHoNSbgH4H%RLzHGoJBX$B7Y7}c^jUl!v^Oh)HWfyYGrOOW8h+l?Dj z?Um@0hhi!^u?_M<&SdeYo1+g$XvpsTCL~s9 zcKd9^q9I_E4-lWtA;QS88jQ-#3uOwZH4%Ej21NoX&7o~X|L`yq&4oghbjYZ-3H!A$ z9Vqp?$s_HBVB$rk_>Ue50ve7f+C~DH;{L04yb>0N7;ZicP=Mk=041O_B40(FGiD?W z9(?8sOhyk{3f2a6MRCSkVa~Fl$Qv;cMU)a@B*T^%yE@1yGzMC<89yvPl#d~)*xq6% zi9HyIYelZixCTi>jIQxx_aGGodKOZ0zws@s#)zTzU?Gl+|5K#-MtQlGdXe9hapap&5E!MT}m2Q(5qyCwS}e1lT9!ytRB6{M+^f;3KG!)2_^4V z*Lj79L!nSIkOp2@b^u+4ki+fdW4nZuk@mge&y73{K)xZ+@TfDA)-H0mx<+?|tjNH5NZ)=YlASZvkZwTI~42h$5xM52m=BGd>KGpAW+z$)7wy-pQ|q_SGO z5-LkqnJF~~tnj>wSe8eKhFkRdlwm3(7*;b9Y&RmFR-vmyoFV`*j<$GS*p?SZX6=BD zDwWp)ToS?yJ5X@#ez&?_s(S-@vdFh&X9@Bzv4>>!=NB`+wM5_z<3A4uTg)dGWo*EKgS@slhUfte_;LBm%CWBdFSz$P-M4=Z? zrW0n!!SCe2;=|V^8(^?j3%F{;euQC56c(;L2p!cHM-fig>}uwRNCOq$4|7B%ucR9h z=L44?w%P)m@$}oH9X1!8<^#nd#7gdvgZQ^iuQVn^YqnC@;qDD1R|N>#0WW~|V)`VA zBPuF4%xLJPI;t(C9@znb<8Fqe(=BVec(je%?S=UQs32fR8(&K8?^vJOzl+U6s~DX@ zm`_ClxBg6r8Yk3)01=yvx*Wt^*jzSt<^K$y3z_JOtbce@kjjDw;OJFvEyp4n&jTB@>$5@zQUY7^87Ox`! znwWq{#FOLE=;L13R2aBd;3tH@D?+r440&(SHg^pst>oTXt(m0eL2oX+nAB?#)E>}$ zFvN)ixwJM#I8I6C6A6uy(G{44NG(^S=(ycx1v`i+D@+z-&FHr)GqW%>Fq2zD6&D!2 zg@b|r9{>)s!D&lN;mwj5<<7l@5#ba$K?>3cP__fGJV6T!s=8WtdZW&~n)RTv$dJLp zFZnspzwk?PD6ftB`^D!4`leZ zL5~alt9+um>=*j+t(O0LK7Jxm3-S2{K{Qw^K|{-@%msVQ)bMefM&>t!-Wu~(_KrrS zujC3AfWf2FDAe6s0K@a4oyNx_f(rb#XjJ-m9x_5IHy4S#h?N5=hSN%f>eK~C{Xm5} z(j$y-gixL6hp=8E#sgwZy_Ujmd-g9n&;K zQVnK*?2f8zON#Xw6*0r+XzoRV)K$_vr`L_!+Nl3M!C^hpN3y`UA5=U1}ou zO^VVRBHFc4qMD(vT{|? za(93!la)V|F==%||AtRE1SbW_xe+%DzpXNBTq;qHQ*q$9R01QbEk6rWdq}Esk$RYN z&^EKrJ5dqV>$#{6TRY|InAF&#U7?;Oqcg~6)7<3gVKS>>BN{5=D#|(~CZPvhXE=yP zI|qJs7;4ER>Ldi~AfeE0NKK0a04WOZm8P~u7yIF_AY2dK2fS4BiMqeit!2x&LmBYE zC_a7Y?%`~8fmaD3F5}SJsGkk2HluB*w!Dbg#3J{s^kucd>J`&i{>KM*xqx60mJ^{) zg5p{m|0Bgp5S(7bYzV4C`2avJV(rjV#u$-G?pUfosi9fq!kITj8RKZ{F}}$kaA5k8 z`>o!6>G`pRXPH1nL0+3bjH(IJF7ip@Q`^FDx^!b0c2lh7TxTjQPfEf+ZWupF5@Z8G zVkmDZ_||c9WYuqNj6xP|TW!TIo2YHFVilC}`Zo2W?^oWEZ8$L}N6oR;)^P}yVS9GF z;U(pqWsm$8QNbD|sk1=;)T+XfQw7sesz$2GFe2NMvnymjzA&lFMXycKLh%%6p@2uR zTY!PsZ1_yPvL2Uei7MM0z?(?&6ywgOh%FT8EHK<{p<^Uy!k4Ca{ z0p-L(=x*a~+$8_nQJ4U2=OL9sJ|xP|VRdQ?e~{ombfm%nhlLbu611o`2d6T5o9a)A z^a|$*PPu5t5u$BDG=va%hCJCKZ*@g+2eA@fFwL`29d5)zd_I?883seh)KjNi(M>TB zHt5j{SwhAb5mEL2|6Z8MU0_Y3OAVBsq!~C8tfZYrtT_?ppn=5Pf%OYgJ?M0CQgq4N z3MW@~By~$lf2?sP) zj!*0}7jY!jPbHFDKCB!~L2LWN4lgJ~N5}_(vHJO}VES_Ngnm$_FKci{$Tr-f^Eg{n z3Td%U@I4?eWn&9KpNvEf&cnzNtcL?)()hh00*hqD5C}yA(;7KsMA*~i$F9%M^t4VN zY`2ZX5FtR8Iy8HPz!>5Tcj6612MOnRNZzXK9O-C$$U}!hx2&-%Fzv}ogmS{|WkHs= zXnaw+ZF?!My0|Np*yKSIkP0sg=?Zlp&@8{)(m;pvC zHY6GMursSwYiD_-YQ-(BMnwhjdP0I0O;iJ?Jw#~HxS(GkHGyOjH-iH-BgxT-nrmOT zfImtB(AjOR@>l`=WIpI*n>`tu-4uV2vL9OWg}IT6xUJ&0CxqmNZyWhYAsejMs~B8L zM8s-^Nl*l&1*L_@3ZWo>>B8?~EAB$w>dy#gr)I1b5?B_EKw?e-(}~|*B9~v`hGSiK zcu8Ww&xrLYn_`}k=Mm16R8;L$!7q z(t=RvgX6_<4hjDMZ|1;nzev$U;6N-ec|N0=vm> zr0u@n|6sgk=tUCnhiwA`Zi(aSbw`kmoQ(4fgZaqxX4w21JI}#H$-3-CLEQl?s z6Ndr$&5$DoJ6%6u7))BEm-^>s1 zUqIffFAs5T7BWhpMk1|}Y(<15v_{9=qh+PDitrQ=r%)pL5P@*Jg)ynVrdN$^DKplL z?L|P8(gV123RT=u*Q)}rmbOCSQS`AyIR&oFt3TS*Di^wzZVS_%jBChbA_j}@RB?!H zOm(R^6g0nxe(@;Lm+1hIwzs)!vlqI@hZggOBO1cJ## zBI(0M7PKt?U-(a47AS{^217K1Zz(7TVvJd4_2_1qu$GjRZKt%r+##>S3LXwXDz>q# zLLV1#Xo9p3RyW+p3>9U-D3)h|+=G(7(lewgm1|qQ7$Yp4LH01bVvZ0QHtHw9viz5U zRH|#4l?;%>;8Ei3bwj@pkd}_5igjXZjkf_sR)RT3-bU1YP$&fSkMl#M4QVfB6QEbI zlL=D~URFJ)$!cxirCaM(T~a$HbxaZCl6@8>bSDv}a-{8PSj!zDkx3`^9^C6TS+$1GjXo77BxHmJ z`vS;UOVwQC;i8Zo13^bp4S`PLjt5??y}(8J!Ck_}%@Y%q6bOgcBeFp`ZxvFRL$C{6 zSZ-NlfFSmUE{YGfp@94jh0+{KB1Z%0<7;2;e zKe~`GO&l2rDSl6FNySq|Ye;u<;^Xn_En>onk%6yN0gF4@VV?po2xX-TV6hdwIv;)^ z+TtQYr@e+e&^SamLV#}60ENw}1t>xz0~uuNz;qxXcA!ikgiu_L*k4*T1p*~{o}Hq$ ztz=K}aA&~J(qVy#z=hQ%?-Ev$XmMO-z>Cez4U?z|=@f`BAj%=X*04)}t_b8AEcX^C zWSd9@Y89z=rS7^g!zY1=$0-d0gK_L(x?JwGx-pL%J${iZOV^SfEJt^)1if77thQ@S0Fcy3RTTFbiq)Jzz>Nl zGh~-FX>vu8n?J$ye1C2L51GQtn~hgb`&;CGS@<0h<(XQ}&~VPmyorY+DNJ*K(+N&n z@-h8Fh+!_mZOPKwp{iZ!yw6s33XuIxc-k6|$-VdJVX3E&-l%vf5~`s+m!$m82Xb zfNJC_R*pc27h*`oAjg5fh3P=M&l7T4y|QY#5!z*}+JMyblF2ZPhbt{{?#AtVXV{8Yto=VY(qkGvwS8BiF=AJ%8> zl_T63l5yFV9<+6AOFINDB2H3q^2J9d@6dgp5zBWX>$k@60Y%>y8r(LTZ;}5!Nw-sEQ074bbv?e zbN%x}Rd28rw3A{q(NpZF7c9C0?3hE8LX!Q&mMO?mP)kb^#T4!phq~DEg$^eP(CFe5 z9A2DTw4Ww$EaD8Yk0@x8d>CLRVN)}`l5t)H^ARP}MyzJVTl_c<;lr@tyLI{uJ^0_S zK^T{Itu?r>fFQ!`?C7$n3XL~7~C+}P)Lch8n#xsN2E{4 z?e&qg>gA}9M~J;c?Y)9ikxD4x>KV$FBxZ`+#pYC0U}R#ukZfhlC?vkDut;qPM=uo0 z0l*ou(*K&cH(&*(>&GpMtxLGl`0gIn{OlOiq8Q{Z!7URGU>>0BE^p8%zqJt&*Zo$s z9TNsLQp+Hx@{_Vj!dmRW#1j~MD3Ro7krtyAQI`KUf~35m!tUyd&ISUys#zK75z_9~ z(~Q*QON~KSdoT7;Cb)2Til31~SesVf5W=lqJeGEHj;*rwVqg7F^h&J`HN>rwd?+>( z={8ti0FF?J!4DAeJ>$D8{UXuQ9`q24g$C25mDg-6!PLMzM;beEzOcs4M4nv;X+|OY zo)j9Pd;oC-wC8d`xJOb2R*`WI5e@DUr`3z#fr$UBUpN-?QD7^RqO4oWR)=6A%fR-{ zNKGSEf$9+Qg3*$M3NVVGDI*6=L}1i_JaGxSM)r8Ozy+WoqqZXYfEeG1;83ZjPVsy< zNuhje5(GH|5L}2Du#qtcw*t^k7iKN$)Ff$Hy#63(kmx(qykR4@S6zFJfW_E-N!=jz zRQZB6S~tQjYQv1beaLaotAH=$>_PG+JZyOWwU>`N-Fua(CWeDonPRD!x!lPi(NEAH zxmCbu#L^Q%4Q~@Kba^YoGT)b*m8S#GFz-6jH!WB}; zJtf>Y0-J1H1!T6Nd~0o{JFRY)7m*yVgTN#NL-EE>Eb}PG3;<7IqDJn3ASH@?XU--C zwAxxkkEusbA+C)2gvg{J)tJGq-W8pd7cHVK#YMFh8*en&-EdIEoCqXFJ@%hcWdZkc zdZ6K93-MY&4n@9_fY8UG`;tqt*46s(P%MgQfS|_ds}vFcth?`xpQ%d$-Fe}QH9#rt z0cw@e7Frm#C#0!ZLjq1Obad*w#67ZFE$x+~YcdB_Z=~$H57Q3hvNvFZaYC;ndT}F6 znvszUD%vXI%YufujdsD4gI+Rp4C1ppkamQM6%k(a9Xa2tt#MwwutaP&vmjL`^BT?y z;Ee3C+H1O$;7e>SPi0jiAV(P+O1+!jYe>Ib4O+Rvoq4~4yh?>avCVpFo^4ufS!mdg5wMUjb?X* z86Y6*Q*7V(_-L!Da0fa8K}pi%FjQ5zwDgku1*Qy{7~oj}{GD10H63(yK~mGPTW5n0 z9)SU@j=w@oB11IB1rMnQsu1$PgKnZF>L)XIBPecu-q-_r3n3z*4nvZ)yx*%57sT!% z@7AnNaFkj;A0@iAihQkL%8al^68i0ss~q$7e(kJ+wEjc z#2rIAFP|0CnNcFkIYR8#hGIF$c9@5l%_W4gg6X7g!IGXI1`LuabHwRZ zlB_+*SF!GaeGS{R_Na#_)^i2$RXBBcTy~lCT!ydMVl+$zha#ea_+bt&{ls@wst3I9{?aiq!QqvLIePP5IKw-4Fe#!eDV>>caAdgG_nsGW%oO6_`~6j zhNu`8Uv73O!pSct-m z3n4y?-l&ig18ypB*H?@JMPrMyYp3K7B@scDsph0(C5B|f&W4vb;EukTwIGhgtoUMx zPqPRTeHiK+ktOml#TkuXJMe3+X3fGGK_(};6aY*S&d^>f!l5C6k52HnY@l|IIRS`h&aM!>p_pL(6R3jaBZtMT7giZ zZwCxO6h&ool|tVlGQsC^1w5>mek)J|t3|I}eNAzah$B6qD66VN#CS9=?7A`@_5HCB zU>iCdI64bFJc@LySOmos2H0S+LR7;HX74fo9Eh194u@9{P_r}oj>zqU_*1?D*2bae zu!B=22El1fKEY~So???Rgp0v+We)h!nk6LV0S%3OpZtn^glyq5$_B6+5EK=;PDOW3 z(P(3P`M)a-CYG5&Xlr45`0PQ15rSHu#OZNuL$F%YLL*bN(*2-(l@mn>j3O=|pm=Qh z^8RRNk|`S&4rt6%lzVVV;a-&=rU1@Iwn8M^hE1_;2{d42F;voEDMC&J$C3pBdU&zp zP0tH2QdWmJMQef!jCSu(p^+!%3e^)ZCt4l^_74A}T~;5>v1GNjV;rUX(;>V7EiFyu z)CguaoEXAy0*uKLf>cDdQW<{WY15rb{8Hz8_o|2$yf#sU?`+=?XoNYc`&f_yS}pow zLHlBzM;AEdK+7N)HZ-g`7PEv$NLLTgG#mr)I|?F#=!gAd{|N|4LE;I+G(x}rsP2%l zOw*Ag951XCPN69Ujj>kAf0kYB32=}X^97+h`7n7Fjf?Fvs=kKo#Pe}MApB0Qvew`@ z^G#pjv>c4w&`>28i1EmQ7(sDo$l|u4(1qj$ z=H}&Oi!=@nPVNvKS~&0Vt@yMmqJ66WB{e-w@Dn@(Vdn%UkLajug1UsoSyiG4pNKuu za0d$UtfB#Vue#sF*~9HZU(DG$B_|Y$5g1dr`m}>Ix{a_C_`Ptt0A!A?w8%>l%I*mJ zpbqbG1%PQqe47Ft3s+hwgIJu^mf^cw9E@_aGf*Ytw2<%i3lbE6eFH1Xh5wb_g!J4G zTLk<@o&dbk+H+%96g<^xCadUwj5t6}>{18?+T|0rVM%?#HDta;&=msi+tO z)Q$qIl$s~bn5pU+lOI_qr768*b0O}6D?1bvlDKL$a*io@C67qT5xgo^m&nO!ojp=_ z4_X0_r01nW&2zT^V|qESgJN={_3c5gO{**_N@UeeB&o{A%?VD4?UAbD!cmLRliwcl z$Q%E&HC_mp_(1VlEMBOM9 zx!$Aok2(~EQ_V$2C`mDp3@blzO5v2JU-1T@BMD<1I+uQ*yiaX!)VBs`CIXLa+#K_n zn@Ej%qRU4CCu7JcY>2$V7DDzs%q zg|gLwr6eC&3Bt7%zp10$i(p;Shlnm&p3W_h1#Nt=X9(kuD_8 z&W_HB{gVF;pbDRwSRQ0pLGT&?{9`8fMQ0eNM@c*oZ^D-Hcu9})ohe= zM~m8Mo~8nOU2y}&+TR5TDxr+4xuis>NpVx@w)h411ZG$ygJV$$qnN?$x_EM3oUo$b zumCIIR7N|4+YU9qB>*DH4J;_Sv=XBB=8o_UWn^TD;|+8~!ehW~WM^a9lD;gM%kFr9EEy z;bSsJtPlo4XqBrh6*3G)aGG>}-~i;vk_r(yJR{Z@PreX5=j7~8ADShA*`m+FPGIsR zKv`L%P><5)rLuxZ+oKRPh7#sc3JNM)#b1x>gdA&l7J=Rpr;luhSWXMPtX9R#nw*!D z9{}<#Cc60SA!q;)M})Y?2K{f;-An^=vtrMdMRF-=v25*3iLca;imM~QsSR}r?I9Ym zvg*20sH?R=x8jF}^W)6k?jS5zg-8_WKS z{Cf;*jqc@Qx;Qq6(bz;ai&0=`=n;CTH{nMRZ^;VjUr*&?A@5kg;l~axcJriA`rtHw zc24)4^z0nGKRpzdk%$gbzSjkobJO+Pv;cEgsHd|@;swEebIDOFmr%?gRaix`5U8OC z#l3CQLK8{~GBu%<7Is>II}<9h={2dgMI=|0vH&`VrVr1~gFaW$TCsDcQBv?+U>V(D zrb)yDTpBrrqRf*5WSE;oJvVt>A~zjLdF_t)w4mBdE1LRPTO9I&)K|}M<-;YUMu`8iEv5E%1@+i)Uq7tbEU4m)bk3c-| zIZ}ny^+++jL=u7I1w`>jU?`@4|SmY~( zgr`@448-n<^?)N2^U8^DFY-na`7XHGYS-_^={!WTufXg@o`FEfdC180#@>IC2Tv<5 z2sf#`I7ngZaAd}Qm{x=|R)Q3Lr7(4P68$hI6^NaYA%#SSb zW^%)aof4UfFiA84t6t-OOBD}-EC4z$z@g8JgCyc3D*fnE=`Iv<4TV;tz{h0jZ-Ji2 zMKBw&ybv*CS@lJFPwZhKxypG1DPyu)wf6l_y>Q@H;XNSk$h)A8646B$#+ErU@u@M~ z8K^!xN#72ymkq%|^pi~J2VXUM;e z!)DMRDuWoNAC)uA2GohPK2jd2%yX*?o(^HhrLZ$89$%G)K+4p@Mdk1X?Hs%u0RRIR z)5j{fAvSJkwJU+ch^U_;?abnDC)vcunIvLAF2odKzjs-D;5~yPAz9w>k1-5nJE3g` z6ji7a5HO8=S&q2bnbKDD2o|aU??DeU2Y_D_1%%ED<~8jmAtstGl7Xcq28_@vFj6wT zJ_tD&H}a<^ps=gH{mvTaw|m=V(k+u`8XLTG9*q8eF7Iy;MLw7rC?eV6bq)5gG`&Q&c&(QVv_&A!zCN)@W=JC4{Tb!Y-{pF9Nn8Vq<8I z5$SCYVf1L|a9Pcu+o~G!XfOfUSmz&uMr3w}$`_$Q~$ENCw2c zONLrxE5#|^V>EJ#aHLd{DFh&s4wJ64s6+36D$T5pT?k7!q1kOd%r`PpwKr2YrlJ@= zPzbY!0Fw;CuAtw433&_;65=ry(1nPg^Sive$9Q#Lr6JvmjUaU++h? zyO278gWgG&3N)lJg{uR>rE1yBT}2?@71^O4`PtgO(nc(&g-wbWz6&;fQUS>@u4>Ii*!f;=d6iBNz!L-tT2&Lv%ql;op`lBq$F34r$QdpdAWo?L{wb zTX92Rg{HlWEsAPoUHb6x7@ZB;VImqNBu+R)phzjtQ^@AT9;#mv0@S%_Y7Uq9oZ65G zTgYBS^qx%(M5`PU>R?a}b9)S7qZ}yuvDM3vVH+lb!tzZT6!KEgv-+`0v6W*M8(5$W zGePW0+WSOBiWT%J%CbIEkBEHea#Wq!mwU^-**GlY9BL!Qp>-Vpa2a|HV2&y+ zvO?aK_7_Y$eNVMB{r&IL5lOn%8psM zeJ)yIJOZgk>BwXe{FX1L4L2(&N&o{0nR;w~a3$*Kh+Z8Ek|V9yHe~3)rje6t^$5zt z#I~tzJ0^E;mC~j|V$WpW(-QI!oq}!33V0UgTeU*n-cW)z(Bou<(+68^LU9k0QS4=3 zhSjQaL-q`jBqn68!(7qVBGe`9^2of75DN=#dKVsC z2f8|1?HX<`TEQe*SS&;Mju_r#2mAv!k_Wre$;pE3Dbj%g+T)`?I3{pGdVgf9zfCH4 zqyk2yr@Z(ZAyE?>R6Ve06B@YO@&_}1jPduCaJwato5iW#MB~e6#Koj>8^WzR5~N( zPV9$_g+#>i9*N{{kpSty7Y1ilWb4JFRimfM&dARc z=W+ywiP}SCY6U$AjdjH5QI{>F9i_h^eBC0GI#t(vVDPFDV6jZ;#e@W3Hpfe-_4q0N zNyK;f&dQ`!qRjaoeiQ_mwY3Wm3ldI{nv?d0g0t4Pf($85K*?_D!kWM~%n$UG)P=f@ zqE*mNsmUUkFmN=fVoTt+91w;=8G?uzC%?E0n0lsjYezb}h2;h=I=~KD*})KSEG1?O zXIfsWNFJphF-l185Gta6@?B-EszG4}Ek$D5##|+f0gfAT`qdVxvn)cT+8dGAmg>t1 zXmb?5H3{&LirE0#=yxkmF3iv8b0G6YYy{W0kkgI%=9lV=0lE`*S_O26yixt=N~l~Ssg=K|?oojx#1k*%Z>fPaqyV{t0Io<8 z$7+um1Q&5c)m?>c3g*1hPNOuJ520dGP&kbvZR<2VG!D6V^oFTa8!7oEy26hWg*^sP zFbkQXPM6XARaPbyd!S+8)D+D&Y3UikWJ8`N1e;;FT174yOtVoP^-sUQjqxNBirL*D z`jvG@@2?Twtr)gaoooT$E0GonSP>uB4`D4nGq1=v!!fFU+#(tQ7DY5G6+=*L5D4nv zct*iNOiEH_V#W!%xrOb%(MP4=s4Qi@kBEawQZ9TNSpv?iYz;tp>&v)!zRh31TYlb9 zeXFpuM+k?cKIB9RwMHoboQy#zR()*`fkMyeV_|k-Sh^gt`GxhcOHH}6x_u<1N2Mr< zfrd2%nFetEfX}H{bqT%;_#8={7l{OJl*eS7^us5{L1A5LG-#Bu{4bstScQCE2!b3O z768=5e(wKfSwKQfs82ds)UcYP9vWd68Th0~9waH;3s4=q5@LRJXhx$xtx}Q>6OnJ| z!jmi#%K%IeJa$FB9{xiG&1+YD{(?CxV!$f?$oTXTQGi}jtRN|3RrlLL(aQ%!waN5& zujT&Y=fpGZx3Wg~A#X{ zUCh&{1NXn-cvLN9tWgG@7v~JbNp9KP=;<1}{fq801yA6Q6&`m#c`{*0z~X&)JD=RC zhW+DxeCdo9>yMO+ilq*L%?}ZKlAHwGU4KgS;nMEg7-YMp|-u(z44AmjZO+>>#+{W%($%W@XrF<`Vm6cVAFi=!UqCQdAFGSuLtn z9{_A8HWajIp?`-n1b7}U_=u*7V*)fch9xWZs3ni@f_4;71xno)JddfxCBdgSWXCIC zC>B6wQF!s_C*dF*^RYk=4-x-XHv6!!xnMMGQywOkaI zz^`lNHN(%C7p~@%73!FSByW^(Mru(wCn}FFVk9g^ipHASeyc@wE=Jo`(T<1$h4Z-7 z1j?%DQ$>d(c5^WR(4*i_vGqz{3xZi_HU+#;7?bKIUomPuv|6+^TIK5iivx!u3I>jF z2x1JaQ>?BC&#K~UwIaNx`&zvnS|xW#Y1O(*B7>EKve5ex5qdN!WDV+?hugF;Jsevo z7ffAxxy6zMPO zjOrL1shB_U=eGjB7yqus)0`I2Czv6AKvg09pz0uzF?}VXQ}moO$kz~-LQmBja5+fC zh^`DHwNNB538oO^l8I*{(cK$#%X=@uxFVOIJR`K_N$&z}qBSDoWaJR}53XD#eo00_O()yQnIJf#42jDM&S0L^ht(JKBmye_GxjR0TbR$1%UJ;> zM5Y6pZjNqvF$K~WW6&#=JTGgx-{sl^;+6+zF5isNY-P z?SvpCn$p_j7AZ&YlPJpY-T7Ut=4wktF0AmU>#8zdTz4#;|=jev+#ku5SG z+oc|tHA6l!pE`&ak6gSU$N3N%JF>O$aeg{adQX)w{i4=9Hyz`%W6MH&#!4o?yRsol zABIX;#3;-QIoS>KLN+g>DGR?629{7iQ1s8poe;=5q@%>L7JyI={w08$usm6{Y70y` zuW=7n*9jvwys+!~xhf)WK|9Jvkn~2;bLt_M3mx}}VBs6g8bvo==)9(#+3a@qM2Ac0 zS_GYOUKDt7Ar9Zjyid&se4%D3fvW-=_dhy?YHtMq2X~7dJ$htx^tX~rtymoin_RT2 z04N)bPMD26LbK2?DA;^syGl7R#=}*qa15=r7_c5vkYL&)zYWbxb#6z`LDj`?@cc(A zlz2Gn+L7&Jn}Gq(qE#bb`Xcui%m_buW+FcXZ@8>XodEyT6GMiol?;4%kVwRYZbDnj z{#_0A0J0Js4ehX5I3UTA577WcD&wVe+;Q9(!bB5jGbPEpZFliyG+>nJiXJ4)5C)5m z`NQ~-PjVLE;-veXY%0z(QP!Mt5Rj*x0#gC{G}xnIAhE-wi)F0#&?W59|bm^Exl(Sv0q&6uXyCtQ1jm3m@m~p_9a^EdZ>Qlpq)c zKm>dJ7Q$foyZi}yNz<3qOS%HErFzi)#+Pluj&O#^dk}oANGYS~I;Pv^pd6>rEU1mJw({%Ahv+- zg9#V1VTM6U>f~(eD5%r4ubB&8d>?B#xbjZkprd=cpg4@MI8X)J$4&K8q zSaW@fxn;%;6~f?m*U`pANAyPZ%3Ky~zt{QS=>^9`v99A1nGB0dm#8NzNM?#zj*^SR;G-{7FT^96`rmr36IW`^> zXMs01$RU=_E@-FureW+4Yut|GYV4SXtG1{erd=a_ywGT!WzH=D1gvZU!_s57>W4&x<^7VV`?E7VJxFQ>-m*m;a`P2*~88wp`*4Kkf@ zdP&o!>m^MGS)3Nx;x?VDu}vq}&~S}>z@n}+dP&n4*GrP+A>AJcP+END7nWI@x7Ko&IZAX(6KBaj76Hv(DEv>%OK=8CttxC|r3 zYrKR%+fdA7yaXYbp%|c5?K_$77w$NBgVitUu+>YN&SFZ&P7kN?61aT%Z1W9AmNb2K zoCi7N%>hDC0!tExYm_*h5267(3FrXfIdqe2oS>I9eGDaIU(wEE;%qL)f}ACb+UI&n z(+24!O$RO|W1DV^+jOqR7Q|<1xMmY~D;A6C!45tjt*Gf*p_eq@31pq-JAo`|+DGE) znW0R9Zr0>lti~7TQk)MSr=U%gEZPZ4FKN1>=_M`r4xoS+tWWrQ@Q>jg1bXEP<)9)<{n(hsu?o7NcTZil?te0i58O&>#@u@k}0&2g4@V?kiu#+$|6 zMOy*%lBQEeFKPNZdP&nm6D4C`+!pu6xoRjVN)~k^kVQ@3Qm@q17}QIeZyECG=39m= zX}UP*PYLepyGnQQ=rdQE@bMoRw>8$C)n3dP&pQiT4~!oM&6OSj30Rf~KEG z7Bt-rWI^)@t+ml~+mJN^AAzR{?JZI^Q7Ayx#Y-T>Z3z+m0D$02kOfWm7g^9wCL*kB z=;b=?@NGIR1mVeF34(@nll$$}Yb3oBW+=$M;T-ZRJV-bsM6|`@M-nCE(hH)Q!{G%s zT$&YPPeTj`&__^UQi5Rs+Im6LtxdMkbZe6ZO}92#&~(Kx4k>%QsbDvZdWe7x*N~Aj zT(h~2*Vx;P7m+d0Mv3WyNXgjM%W1p>-MOI{x-3O~NSwPJ#tI40X%(7JD_PQfy2z5I zJ)u9{eE&1H-Ogi-tDd-;+BK9jI22SVNpcf_L&z4HMw0cCrv0IpG!0YeB~AN7FKIgP zDH%J4Y;pTTTrKLhXsnZMQQmLbA(V`5k~MCVxEfm!4Q#EM=9`c#Y1$yYPSY=>mxTEW zRX{?cPS|Gben49AV1F zdJ5_^-&WqCFKE_BvY_b(EDM^hD6*jG1}qDjZosmj=>{wdntpFt z&~&Gk1p$hJ(DDmVpoPpHk(&ses}p%u5E2fLH$fIO-O^>jkOL%X*cyg#QE){B;h49A zCk^?+ye4=cy++f$U)E^4_sfE&d%rAbI&x(}QzJnZG+!&(5qc&KZ8O@_5E-Zo||=J6P>VTsT(WqCIppDyF=PU9trCJha=bFGLEiF3EZSRr&nT7{;I zj$YFA{q&ONwuG@ob{Z7NmM|7XKSFD|xhWw_n!co7r|C=TCCvxDvF*0FFDb4T)j%-T ziGd*RH(kZ`l4PHgGY`>&&5c)$4C}B^`E>8ttL=1c9vMt%(T(icz zh6;%G=B9h3Ueer>kae0{60)S}N7tWjZXd`xO;_2t9TKPGEEZX%X(!2orV~RJH0>f; z(A1ES1F1UOP3;3&(9}MV1x@V(S{)yYY1p zcQEv$m1{*D0D4K&&d^JmPJ}ouve6);y(6wFrhw_g^^&F=qF&N`ZPuS|Y97St;Q#T9^Zl8|;4@>)>vIsE}rDgDr-`SiCo^)$#DO)Uvo&@`YW3!0i7vY@HS zAq$$C9I~JTJr*?3ya)jxZ;ggcfc_$QA=2oA9w%8Pa`ZqDVMDi#T%j<^wTPgRjSHVA z`HS$-1i-Ip1xOG&5f`<) z;@s_$H=9mry`;IRB|5I6b_Z8u z3&O}_xJEXo)h2q`4y~PJhJhOslb=E6z1GxFYwa zUeeTApqDgF?bb`04miD}=@DBmY5IlqlI9zku{)s?F^o;`OT^pf$8_E*OQ$crPlA!w&RUndwzFAO6 zN7w?eugfYt;p~iH8X7E)+z>(n2nA-PqDMi@BNu@S*xN%vo3ndj&s0Gk$d9CMNo^BT z5_@*-nA9=3D;0YXqjjKkgu)|HDyLepa`rT}YGqfMTD7vEsZ}crnx3CzK~5rw*;!mp zuMI|37_}U>u-5|RMi`zqSHLYgT6q~$J6RSqwUcE*Q}b6A^q85yvY@H?D+`*g8nU3N z`6~;Wn!mE3>6Ri3n!@K~K_r3#J|g_k`XdPoWrmP34uv7egfIx%H$I{Me`mh9@||ME zYQMyV;@ZV)#ZRu6;QFJ*N|Y>CtOWlnR%|5y^%eid|NU94!i1J}N))?N{j15J)KPk*4yfqM08S}Ltx)-I{qq(`G(iIVE|;lpR%W zZ@=@vtEax5(Choji+8r&RbqJaiVr++{*~Kb7`3d+hh-oAWXY*vW7ixVT_t5+%6ExL zQ(J7G`pCDLf82ib=5mAE=dOOFYw?D!ReJQ#F%!>S9{#=RFAe&3uJp{ly=8Kz-#n#H zx%tm}`W*1wxV6%&mF~-3)g$3==e9}rpUUAdHBg2xWSHE|gZOzC>8b3My&*}rl zcG*0z?A}VR{n06X@z}c4ULLTz!_;ACo)|T)_t-COU;h35k5%qS*Dzbuzj

t9b@{ZPg;*Y7TQzV_iW-E!aCx2fLGS6yDG zX6@w%oAg;a_Q@3~uYK`V-oqpNJ+S42jSqK8Zd75)@xDKGO1`aGhwtBc*H*5zGHcd^Sx3%V*LF(|rr-O+@?ljUoi?h=)OKGU zTAy<4%l!}k_QUiu*^9n8{K^eKZ7%ao|7zW~wdy}(!Gsgnuiac@^PA23m*02pz+)d) zu6{+^foD6sQn}{gHH{DCU*2@7|Lvp_p{(b+bG_Fd2(0|E=d5>c%NkOp)|2_Z7jl-qeqht1?Y}uZ za8`|V8|OdxMzG@ffghFn^s!fmJyYq&!wc@NcjryV+iY4?+`DT+uu4P|fbu(b7SKNLGqc6#fa zKS!2oHe=;yA9X8xN8P*z4Hm9{e(mV%225Y)oO8$ehprv%tDbV*s=oU^J~;REPseYr z^57Fk9y~DdmNtDh*RP)6@$|2o?TDo~ewSo7yZ}DcCtWhiGdKX^Pyz`h! z`+m!)xS(mdcZap!f2zm*XDin@^VDGP_em|M99Xq*_J$V+ZcHD3ZrW?*-&%T4*?F)2 z-TG^N=Py{{fk9q`o4rM~>T+D+Aa9&Xn- zv-9x2@0C0K@VMj#yL_J%dvX3ZE4zQ({m8qoT~n;xgJlP%c3qIR{o!g$H+{Tl{sSM5 zeqeK}_T7J9kyLy_V#r>(a_K&o^*mH#^UN0qcYOX9=c{E)XK#CY+t|O3Txr?iZZ)au zvKtrvy?xB#jem6Wtj;^KyVAhNyDi=LN%A-AJG44A_LY&Ze4qJpjnB{Zc)>d7orIU? zW<36zW5t@P&nG-L_2+><`Mx^eseg+PJ}q9M_u47vX4Wd6_EoQqo!v9X?saV6Flg29 zpUyu1^{xgtRNb&DuwvGfmD{E^JZE_|<)u!2o@wTK;_8pe_}6_ox_{4wvu>LB$EX*Z zG(TH*%;gI@=Jb1~>)i6)E>HQo`LJ*Pn0<5Ast@$>hVCr>&fMMZ6Uzq_`{}ilJ3c#I zdt_SoaWC#|_Mm;}qPAO~{$T!BE3@u={ZxYzTW_rN;!`DmdvSEvY|qXs>rEJYrDOZB zdfVUXYJ0ZQ*}MECbeDhL7fx( z^V+!9pS^zMS!aG_f2Aj0m^-n=#@5$fJ7L|QlfL`&aQRo>DIOU5POU4y`KxOM_w(N- zT~@91N7Yw<*J}0WJL{iV^UK$-f0iAdvg)BbT%Xr?Vu8C}^%`lV|FnNIf5ROscWs%} z@6LM~CHgXd-+JTu&wo5|z3aq?1^e@U{H}MYM@x6Dku%=D{`mnVuR7D?gDovKr3BMj zlxqC-ud6@Y;cEDFmEG@7+wfwI8bc5Kx@^UQ(AeDT*3J*yHo$Smw*CvY=|J1cXGflPg{3{2Omw}vaM43yf&LY`|2@!?{icA9ji?LtMc@A72MN?obb9Q-8ZaN z-&ZCcTy}M@Qms~OI6tY_tE1ki=3esC3s)W;v46)44|+#5y06oKJM4M)-1^5&+osGI zw&do=XD+y9fA5tuKVIGQv+w?%Io|!$o?!{L3a#$FJMUp<_B*>PUGZhotv@#{ox36& zY+7Yg&)fFgmDqXauwSl8?EF>AO-;tuZBTOKC%Lb5dGPhK>o7O?qt6Tq} z$0z+^Y3zTuVbb!?nmw?!@|=g-tZm-ev1#DC#&`YIZ_LtRb4qP`Z||>-AN$!~^RFdq zYG%A$d-OU>wJz(r?rc(N;M~ko(>qjMv~k_Fqd)UL|5wQa>%7l)Z`8Vt<--qt&Yb=6 zfrZQ7dvs-&_g7Bo-T2oU`=;HyH}%z5YwZ~`_Rie)XZw}SOzv^$;SMuWCLh_lyh5AW zCr-R^s%*U*cI?kN(6QBbjT(-AxYcvP#tReM9dT!7KQZpIqwl`6DR=H`q07gPyZ_F& zuWOOF>+2WRG(B0n^|w>XK60jH-goUfv_AXk_-E>NJN<0vj=Z#czuUR?@gB)dhQC*_ z*@DXbHa<7*s5gDKBdb{Z4g(K7zvkZDC(e}p&{uKitbVt&oqM>)%yMf#b9!Dm`_Rar zx(|GDQ>zC`hmW1;y1Lq&2afIO?|ZECsa~H?+Zt-wV7lkcEh&d~6hHo1{W5c}x_Z`( z(G@C|>H2%EDwBU`@Y_!BtiNWwpS-&G1{K;GQq`J@48F2cE%NE)jj17HW|5%D`!M$l| z?s``(J9e=CyUo6xf7)~W&%LXvJkq}PrUf5Qd$aD$cK26lRWYaLgp%LwJ(2U=V@>n&9;Es%bU!^5h+HrU5U+y^M9Qo|X%Hu1(HRsIQQv-glclP0NqZfY3%`;Y5gLpvSYcW&OiH#eotANxZ_)3hOT<}SK^w#qe2I)AsJ;;qelx12Th*RK6X zJv*;N-=Rr6!b!LF&R+T4hokQr@NVC8~Uq=7Pl-YR&ry< z1CKVV_I!gTb1DoQ^zNrW{FJ`z__)6&J<+J#^cf=;Pg-vOa($)G&iMAcJnikI)yI#m z{OXO(SALL{anCKIMkNiM^yxcY)5c5>-8z5Z4G;DGdT^PSK0TFD<4E5phmPOS;_};{ z>0YPOOX>ZN&D=hxS(ny{ZC3yCN$-U0E^p16@kaCY`NzMxduN~LAK3U*h2puB%2&BE zt4px^$(Pr^wPQs6(hsx@o_VZvLfgMa9ucPzcG>dAbq6kJ=)etnhV8$t*cXd7epqUC&pWSux6*_M2fcdR_q);`{`x?>kx#pR+`j9Kd+MDnyWPC- zNWGogYCL@OP`yKU4t#dhtNoq~-S}(xjc#SPJ$bbIi$}k{&Anj#<`eCnul8s4J9o_r zKJ@t)Ge&LybHll}mTq&{`nJw$TrKsvlKZO+J$K*u&&S;{eb>H}M&stbXaDY#R_orp zZ-V{S4W0b&)#$$T%*lE+8!o->vK1W;m;1ZS!k<>J^sfEm6U*pphd%mqS;w*Wdsn~l zi&J~wTt?T#Ql$Uk($o-q2I5gq2_2a7FK5S5<9p!&J+5SkaZl`TFOE-kSW{m6`1mrXI;Uv}*OMEoMJk?^Ny6l^0r^`@etsT>qxM_e>mf zYmGg#Y8|<9H{eEs2>%L>Je0@uc$=5#pPVQGP+Fj=#9KCPa&&PUI zXnyxy;Zn`^T+?^V^PirWxvG)f>Y6%n$*IS7pM3sQ=(};nY`yYUl^)$wh=N1u6Z_KrdC4Zm~P>KktB6#i!S zp*at&={)zrEtfBP;_UB7vx1)wd%oVU*6ZF`+p>KA(J7y7tNn=ctzTvyT)(Z_pV!_v z@`Jn%X;r4r^yYuGqfYx^`a##6=ND%M_aEF8>XW(Z?fO42Z~SNe)`g?@H(T5AuRouz zXgyH>{BZxAy6=t+uKLjX+sXO%q;LKje@BV&!%NhjmA`S}HC^uA_2nDk@oP67 zIx;lJ9Ye%=~HH|oKbUP$@kVw+;s2J;7QL_ll#osx2*N@ z^AlPhIoSAteUDeG(6i3{iAOTlKmAGVHK)JqTmE>xcBu`@E(naAH0ib{pBv?=9)0W1yeYZioZps|ziQQ%J{zyOxy-?9U#>K9Y2udI$L!Z$zj)Sw)7!>B z)@<*0O@^lhzPI;I*;VsQo%&m@X!K{=+OnPdJhX9M`6H#>ji)|UZ))%EH8P}$rP>emC?TQxn~>BPV(n>W2wy7b&84c`CtwKZ3^I`CP)gO;|9QpSYK z-+pYt^xVC3z0>y2EOzLLh4#r0-!i<%)JiXZdtchFzq&3S`skwamGUcH9egM8rP=4N zUcLLh9kYA%f3V8Y?Jo>GvAgZ@s{R{$-&ig0kvY%ZSK_Yat8RMiiMf-e-nqEm+!5fNn=5P75`ezTN1!~=J<>|Ld|2D49 ziTB<~pS*FzHQ}n=o_OWCv<@AHT_3Dcs^`)THyv(yZOFHNee>HMn|ga@%{ND^x^`Xj zJBwwe^mLRMyYaLYJWC9 zZ(Q$%Z*_cgZi)O2eKNPt`Zc%2`Ep(V$em!>@bufSmuk|i!{TptwI4O}?&o`r-q>!z zbDwz+e*K%{@DJC%Q|0B372m(>i-*7I{OpEjY+ruY&^P&>w8WarKiSms=Y~!D{FQvm z*nMBzv^i;CEoMe>COM53}l9N7y!Q zTGeUpBelX+N3RhWov^{aseR9^ahV|}W^u~cl_J;5Oqt0*Nw7sgr z@b5imN4m?FSzUT+!i2Qj%5V5_V3l*3(_h{@WXa#xwCdOH$1Up~IM&bi(Byi5ou3g% ze`W6RGqvh8t6cW+^^Uh5{?y;F+xc3n7tA03+Yc|^-QQt*_o~ecdVe;v#g7^1x)1gJ z-ua5F+(}QonBBMXfIXe=+_t=U(wgv-z5A`W{q!%Xx2)cAqRYm1OICf7TB2qDKGV;X zop_*0|Ni4wKYh)p6-)9~oY>T)Y!^qbmaFfx?wj6wdx<3=<{s)-Hl4!e&p&=%_{y@{h8g>XD`@WX~j?du6*$p>$=}t z{hYR{^KUIazVG+t%SzmoeD+q$FU4JBdj0V2iJ^7!KKN|n2IC;XxZ3~ z<;Kj;sj}wv^0k{hvhWpGod;eo_eAlmFCVyiPnnMUdz^V`=wtVE-gKhRh}2V`_L?)Q zb@xv*cIK{sv)xyF@9xomo&CoV2OoYeX{&qQ&W4MxpS}8Y*X{Xt-JSV!$=$0qo*i;) zd-sOFhkfxuH($MT)5_m=eVy59$5;K`=k~0>zq`Ep)`Q2NTJ!1W6NcseU~{F!qafReDbTa{Db4N}c<*^0>@Tlc#RIv;X2R zx2zi6Zr2b;0K}Vvg6Hm*Z01DO6xKMJH2F|b>_RxvhD@QPJE_U(4x z-0{X^uhgp9xat+PmUNg~|AumVUploY=Z%E?t6O#1QR$gMjbHdayXzIZ{_OSj4_n9W zJa?#tE127EUWcoCZoDh&jnszQ2h^U@d`h(yzn1;ro4O@x?<&3Oms?hJp17giJqt(g zIq<^K{U3)1raivN5qR_FS)bopVd;n6_LXVyVRqk5886-2spDL08A z@aguWBerg@zWKhM^Ov-~=FwqCrml2de$(xbY@gk*@}OJCIWM+;5$9e zR37$LQukf6cXxe!`S^*=k2LL2yvrxkPZocnY4A|Mv0(1@>YeLe{rld_9(eq_w|7qU ze*M~l^cxRPSJ*N3lfS|wrqvhTlo#Kh zCGTtbWZqA;26wo3$H~MWtAF`evDvLR)M`Ap(VLH1rsRLzF!T4}4e#DFWbcf2XLm2! z@>SJdr#d7RZ+YDaOXaPG8I2NAT-=uIyK#%(1j7Et~xA-}3j)UT;^cUFzLNk5;QVrc~cmU1!;x?@z68 zca=Y0Y+0fIvIm}g?a|sBCOv!h?gtzGdgZRq@-pTIKc97YTJt+*)H!sn&$R55oy#6+ zdvDM5U9}%;@oJ+6X8re1|9#!B>uSHWV?oYy>uwa6*{;RekAGXU zr`k{c7VVDB99?>FgVCkl>0R-WeLwwGA^V=EzbkLg+_3qEACelszv`;~*F5@8Vy%Sa z2Y=euZghtVbB?ZDHfq%B-`pQGS=IXdk<;(YY3CVIZ+EAcKdDq|Y(o29+wu-i$SfAB z+QaWH+w`HVU!EE^r|xgw8AIQAX3;P62GzJVXU)T}Z!P)W_~J(zJoiv)=1r|@9INuk zLqiVV`{?lxS8XW2^ND)jT)p=>UyFM)2X9}J)NpXgo}P+F9e3Q)`_1QWN^DodvwZzk zTa(r%R(tY>N>vjl|GxK+qn@;Phtz5AS@(YeHwMW0u%!4$|6o&l;s~zAbDy>Hb+YhB zW$cGsabdLu4FT3PW%FJXIbc9`mag#qE7I582C%oX%Lt=fiN1oCB+aY%>8FBWoEXVo zB}Rx;iSdZmxr#^ejOK znn2bLVU0-$Yj}xmsL2;fiwMnR)_aWEB>`s3kQ+)e`N=!}mwX4p3FMKBa93Xe?!M)M?IXbIUX!l|5jc--cMeQu zW_;?KgD#MHLkgK~0qqWk5rtKV+4|$vX_-(RD61PHrudO;+!HGEOsI+OHeSS9Sf*z! z5IHxR`*ri&vD&#V^Fl2i_!-6Nm>ragAygG%CB=|gWc(7pzIVJy-w83#Dv-7p_Xe#Y*WsU2-?9Rh>LYrq059` zA(x)>tGU3ss7~vhxquQbtg5^gMEBQ10y=+SfDAAAvmQ)JG0Skekzx1{=$l3}n=PNO%oMG(Z{J0I>rngFnXf*=Um$sW+$xYAYi(gi6Eam|DwA}pTlyVSJH#bXv zLw$Lq)-%wWP5eCe+Dr#L@Xeg&Ma!ohVCOW`a+m>6AXE%* zDy%u50ep>=Hm)-JELmb)iPfdWUwqUdA2;z`L_0gOgcdC0WFuHuck$%=`ndyIvr|wV zSWERe&D@v;#*bN7IwF0VJd@DNZ!gbjVu~KFrZQm-1yggwt)-e%0pN_RUk3I!Z>x$lK-0J7}j z(ovV5FguC76gYttM?@W_>|`(NGPo8Wm3T25-PyJZr(cT#^zUsg2%qG+E}mw6Dt)?l zid{3m)kXPm<0Zh?@t2*5%3H~&VH<1wpzuSbB{!V_xk<>+np1Kgxg3@oT2e8JWtLy- zSdn0>3uiAvknJ=K48g)1Wt?s9kfp{{`Y))%MIF2b*n4_|yw5jB5uM$rOh0|mK)I!{ z7kgb8uG-ug8P4kc$yC~C0{Eg&*V;wHXWHYq5CLArPiM@bV4S9|XWmWrs&V9Yi%53nPh>Jq2^oINQh^oy;~xMz{KNog4&9 zyO)x_s+QL2TsJjxnLk&<+GX`7iNU!-s(t9PB@xvk%9*)!G~~?omBY=GILFXkhn^v8 zx|!40zG1AkD#&01Uv#pZ$vXj*n_0<@ruV|^K8w{9r;V=}g3s3JBJ39DAKWJlo=CNs zeYXc&{SLR{O|}ygj(gwh?=dEYpF14vaXyPUH%YUYld+P=Pw)FbALS(pHCT?ie3Szm zgqOF))=(mBf=7P}njV-dcoZd~>ABG@0>yG-8JhG7PR=XE1Euj$&bP#ck?S3vvitkH~)N~rG z_WYNox@7({@uBuD0X>neF@Z*qT>&tw<9dSl5abpp%f!r`zC07`{ ziV-}FG2!(q9@&VUXErHkD$Q1q`L>#_32|UhT?abn!W?RAu$$|wiJ1PmAUO<;r%AXUi6@PTQzl_k#HNNP? vector for the in-memory reader + java.util.Map vectorMap = new java.util.HashMap<>(); + Random rng = new Random(42); + for (int i = 0; i < NUM_VECTORS; i++) { + float[] vec = new float[DIMENSION]; + for (int j = 0; j < DIMENSION; j++) { + vec[j] = rng.nextFloat(); + } + vectorMap.put((long) i, vec); + } + // Create, populate, and search try (Index index = createIndex(MetricType.L2)) { addVectorsWithIds(index, NUM_VECTORS, DIMENSION, 0); @@ -214,7 +224,7 @@ void testSerialization() { index.search(1, queryVectors, K, SEARCH_LIST_SIZE, originalDistances, originalLabels); - // Serialize + // Serialize (graph + vectors format) long serializeSize = index.serializeSize(); assertTrue(serializeSize > 0); @@ -223,19 +233,22 @@ void testSerialization() { long bytesWritten = index.serialize(serialized); assertEquals(serializeSize, bytesWritten); - // Convert to byte array for deserialization + // Convert to byte array serialized.rewind(); byte[] serializedBytes = new byte[(int) bytesWritten]; serialized.get(serializedBytes); - // Deserialize and verify - try (Index deserializedIndex = Index.deserialize(serializedBytes)) { - assertEquals(DIMENSION, deserializedIndex.getDimension()); - assertEquals(NUM_VECTORS, deserializedIndex.getCount()); + // Create a simple in-memory vector reader for testing. + // The Rust JNI layer invokes readVector(long) via reflection. + TestVectorReader reader = new TestVectorReader(vectorMap); + + // Deserialize via IndexSearcher and verify + try (IndexSearcher searcher = IndexSearcher.create(serializedBytes, reader)) { + assertEquals(DIMENSION, searcher.getDimension()); float[] deserializedDistances = new float[K]; long[] deserializedLabels = new long[K]; - deserializedIndex.search( + searcher.search( 1, queryVectors, K, @@ -243,7 +256,12 @@ void testSerialization() { deserializedDistances, deserializedLabels); - assertArrayEquals(originalLabels, deserializedLabels); + // Verify that search results are valid + for (int i = 0; i < K; i++) { + assertTrue( + deserializedLabels[i] >= 0 && deserializedLabels[i] < NUM_VECTORS, + "Label " + deserializedLabels[i] + " out of range"); + } } } } @@ -440,4 +458,23 @@ private float[] createQueryVectors(int n, int d) { } return vectors; } + + /** + * Simple in-memory vector reader for testing. The Rust JNI layer invokes {@code + * readVector(long)} via reflection — no specific interface is required. + */ + static class TestVectorReader implements java.io.Closeable { + private final java.util.Map vectorMap; + + TestVectorReader(java.util.Map vectorMap) { + this.vectorMap = vectorMap; + } + + public float[] readVector(long vectorId) { + return vectorMap.get(vectorId); + } + + @Override + public void close() {} + } } From c8c4dd0d156f4f815483821bd4cbd742886d4476 Mon Sep 17 00:00:00 2001 From: yantian Date: Thu, 12 Feb 2026 19:14:16 +0800 Subject: [PATCH 4/7] fix --- paimon-diskann/PARAMETER_TUNING.md | 16 - .../spark/sql/DiskAnnVectorIndexE2ETest.scala | 2 +- .../paimon/diskann/index/DiskAnnIndex.java | 51 +- .../diskann/index/DiskAnnIndexMeta.java | 52 ++ .../diskann/index/DiskAnnIndexType.java | 34 - .../index/DiskAnnVectorGlobalIndexReader.java | 62 +- .../index/DiskAnnVectorGlobalIndexWriter.java | 62 +- .../index/DiskAnnVectorIndexOptions.java | 27 +- .../diskann/index/FileIOGraphReader.java | 140 ++-- .../diskann/index/FileIOVectorReader.java | 131 +--- .../diskann/index/ProductQuantizer.java | 396 ----------- .../DiskAnnVectorGlobalIndexScanTest.java | 2 - .../index/DiskAnnVectorGlobalIndexTest.java | 154 ++++- .../apache/paimon/diskann/DiskAnnNative.java | 49 +- .../java/org/apache/paimon/diskann/Index.java | 29 +- .../apache/paimon/diskann/IndexSearcher.java | 57 +- .../src/main/native/Cargo.lock | 65 ++ .../src/main/native/Cargo.toml | 2 + .../src/main/native/src/lib.rs | 626 ++++++------------ ..._provider.rs => paimon_fileio_provider.rs} | 348 ++++------ .../src/main/native/src/pq.rs | 316 +++++++++ .../org/apache/paimon/diskann/IndexTest.java | 158 +---- 22 files changed, 1151 insertions(+), 1628 deletions(-) delete mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexType.java delete mode 100644 paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/ProductQuantizer.java rename paimon-diskann/paimon-diskann-jni/src/main/native/src/{jni_provider.rs => paimon_fileio_provider.rs} (57%) create mode 100644 paimon-diskann/paimon-diskann-jni/src/main/native/src/pq.rs diff --git a/paimon-diskann/PARAMETER_TUNING.md b/paimon-diskann/PARAMETER_TUNING.md index 9986cf218444..056f1058ec99 100644 --- a/paimon-diskann/PARAMETER_TUNING.md +++ b/paimon-diskann/PARAMETER_TUNING.md @@ -108,14 +108,6 @@ DiskANN is a graph-based approximate nearest neighbor (ANN) search algorithm des - **2,000,000**: Default for balanced performance - **5,000,000+**: For large-scale production systems with ample resources -#### `vector.diskann.index-type` -- **Default**: MEMORY -- **Options**: MEMORY, DISK -- **Description**: Type of index structure -- **Recommendations**: - - **MEMORY**: For datasets that fit in RAM (best performance) - - **DISK**: For datasets exceeding RAM (requires SSD) - ## Performance Tuning Guide ### High Recall (>95%) @@ -139,14 +131,6 @@ vector.diskann.build-list-size = 75 vector.diskann.search-list-size = 32 ``` -### Memory-Constrained -```properties -vector.diskann.max-degree = 32 -vector.diskann.build-list-size = 75 -vector.size-per-index = 500000 -vector.diskann.index-type = DISK -``` - ## Best Practices 1. **Start with defaults**: The default parameters are tuned for balanced performance diff --git a/paimon-diskann/paimon-diskann-e2e-test/src/test/scala/org/apache/paimon/spark/sql/DiskAnnVectorIndexE2ETest.scala b/paimon-diskann/paimon-diskann-e2e-test/src/test/scala/org/apache/paimon/spark/sql/DiskAnnVectorIndexE2ETest.scala index ceb7279c4a35..1ba4dd7144de 100644 --- a/paimon-diskann/paimon-diskann-e2e-test/src/test/scala/org/apache/paimon/spark/sql/DiskAnnVectorIndexE2ETest.scala +++ b/paimon-diskann/paimon-diskann-e2e-test/src/test/scala/org/apache/paimon/spark/sql/DiskAnnVectorIndexE2ETest.scala @@ -82,7 +82,7 @@ class DiskAnnVectorIndexE2ETest extends PaimonSparkTestBase { spark.sql(s"INSERT INTO T VALUES $values") val output = spark - .sql("CALL sys.create_global_index(table => 'test.T', index_column => 'v', index_type => 'diskann-vector-ann', options => 'vector.dim=3,vector.diskann.index-type=MEMORY')") + .sql("CALL sys.create_global_index(table => 'test.T', index_column => 'v', index_type => 'diskann-vector-ann', options => 'vector.dim=3')") .collect() .head assert(output.getBoolean(0)) diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java index e845a93789e5..eae4aa0bc99c 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndex.java @@ -45,22 +45,16 @@ private DiskAnnIndex(Index index, int dimension, int buildListSize) { } public static DiskAnnIndex create( - int dimension, - DiskAnnVectorMetric metric, - DiskAnnIndexType indexType, - int maxDegree, - int buildListSize) { + int dimension, DiskAnnVectorMetric metric, int maxDegree, int buildListSize) { MetricType metricType = metric.toMetricType(); - Index index = - Index.create(dimension, metricType, indexType.value(), maxDegree, buildListSize); + Index index = Index.create(dimension, metricType, 0, maxDegree, buildListSize); return new DiskAnnIndex(index, dimension, buildListSize); } - public void addWithIds(ByteBuffer vectorBuffer, ByteBuffer idBuffer, int n) { + public void add(ByteBuffer vectorBuffer, int n) { ensureOpen(); validateVectorBuffer(vectorBuffer, n); - validateIdBuffer(idBuffer, n); - index.addWithIds(n, vectorBuffer, idBuffer); + index.add(n, vectorBuffer); } /** @@ -82,9 +76,9 @@ public long serializeSize() { /** * Serialize this index with its Vamana graph adjacency lists into the given direct ByteBuffer. * - *

The serialized data is later split into an index file (header + graph) and a data file - * (raw vectors) by the writer, then loaded by {@link DiskAnnVectorGlobalIndexReader} for - * search. + *

The serialized data (graph + vectors, no header) is later split into an index file (graph + * only) and a data file (raw vectors) by the writer, then loaded by {@link + * DiskAnnVectorGlobalIndexReader} for search. Metadata is stored in {@link DiskAnnIndexMeta}. * * @param buffer a direct ByteBuffer of at least {@link #serializeSize()} bytes * @return the number of bytes written @@ -97,15 +91,24 @@ public long serialize(ByteBuffer buffer) { return index.serialize(buffer); } + /** + * Train a PQ codebook on the vectors in this index and encode all vectors. + * + * @param numSubspaces number of PQ subspaces (M). + * @param maxSamples maximum training samples for K-Means. + * @param kmeansIters number of K-Means iterations. + * @return {@code byte[2]}: [0] = serialized pivots, [1] = serialized compressed codes. + */ + public byte[][] pqTrainAndEncode(int numSubspaces, int maxSamples, int kmeansIters) { + ensureOpen(); + return index.pqTrainAndEncode(numSubspaces, maxSamples, kmeansIters); + } + public static ByteBuffer allocateVectorBuffer(int numVectors, int dimension) { return ByteBuffer.allocateDirect(numVectors * dimension * Float.BYTES) .order(ByteOrder.nativeOrder()); } - public static ByteBuffer allocateIdBuffer(int numIds) { - return ByteBuffer.allocateDirect(numIds * Long.BYTES).order(ByteOrder.nativeOrder()); - } - private void validateVectorBuffer(ByteBuffer buffer, int numVectors) { if (!buffer.isDirect()) { throw new IllegalArgumentException("Vector buffer must be a direct buffer"); @@ -120,20 +123,6 @@ private void validateVectorBuffer(ByteBuffer buffer, int numVectors) { } } - private void validateIdBuffer(ByteBuffer buffer, int numIds) { - if (!buffer.isDirect()) { - throw new IllegalArgumentException("ID buffer must be a direct buffer"); - } - int requiredBytes = numIds * Long.BYTES; - if (buffer.capacity() < requiredBytes) { - throw new IllegalArgumentException( - "ID buffer too small: required " - + requiredBytes - + " bytes, got " - + buffer.capacity()); - } - } - private void ensureOpen() { if (closed) { throw new IllegalStateException("Index has been closed"); diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexMeta.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexMeta.java index d603250d1ed3..45cf9f86dfa6 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexMeta.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexMeta.java @@ -48,6 +48,9 @@ public class DiskAnnIndexMeta implements Serializable { private final long numVectors; private final long minId; private final long maxId; + private final int maxDegree; + private final int buildListSize; + private final int startId; private final String dataFileName; private final String pqPivotsFileName; private final String pqCompressedFileName; @@ -59,6 +62,9 @@ public DiskAnnIndexMeta( long numVectors, long minId, long maxId, + int maxDegree, + int buildListSize, + int startId, String dataFileName, String pqPivotsFileName, String pqCompressedFileName) { @@ -68,11 +74,26 @@ public DiskAnnIndexMeta( this.numVectors = numVectors; this.minId = minId; this.maxId = maxId; + this.maxDegree = maxDegree; + this.buildListSize = buildListSize; + this.startId = startId; this.dataFileName = dataFileName; this.pqPivotsFileName = pqPivotsFileName; this.pqCompressedFileName = pqCompressedFileName; } + public int dim() { + return dim; + } + + public int metricValue() { + return metricValue; + } + + public long numVectors() { + return numVectors; + } + public long minId() { return minId; } @@ -81,11 +102,33 @@ public long maxId() { return maxId; } + public int maxDegree() { + return maxDegree; + } + + public int buildListSize() { + return buildListSize; + } + + public int startId() { + return startId; + } + /** The file name of the separate vector data file. */ public String dataFileName() { return dataFileName; } + /** The file name of the PQ codebook (pivots) file. */ + public String pqPivotsFileName() { + return pqPivotsFileName; + } + + /** The file name of the PQ compressed codes file. */ + public String pqCompressedFileName() { + return pqCompressedFileName; + } + /** Serialize metadata to byte array. */ public byte[] serialize() throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); @@ -97,6 +140,9 @@ public byte[] serialize() throws IOException { out.writeLong(numVectors); out.writeLong(minId); out.writeLong(maxId); + out.writeInt(maxDegree); + out.writeInt(buildListSize); + out.writeInt(startId); out.writeUTF(dataFileName); out.writeUTF(pqPivotsFileName); out.writeUTF(pqCompressedFileName); @@ -117,6 +163,9 @@ public static DiskAnnIndexMeta deserialize(byte[] data) throws IOException { long numVectors = in.readLong(); long minId = in.readLong(); long maxId = in.readLong(); + int maxDegree = in.readInt(); + int buildListSize = in.readInt(); + int startId = in.readInt(); String dataFileName = in.readUTF(); String pqPivotsFileName = in.readUTF(); String pqCompressedFileName = in.readUTF(); @@ -127,6 +176,9 @@ public static DiskAnnIndexMeta deserialize(byte[] data) throws IOException { numVectors, minId, maxId, + maxDegree, + buildListSize, + startId, dataFileName, pqPivotsFileName, pqCompressedFileName); diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexType.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexType.java deleted file mode 100644 index 36612278e08f..000000000000 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnIndexType.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.diskann.index; - -/** DiskANN index type. */ -public enum DiskAnnIndexType { - MEMORY(0); - - private final int value; - - DiskAnnIndexType(int value) { - this.value = value; - } - - public int value() { - return value; - } -} diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java index 9a89bdfe9317..b9f40dd3317d 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java @@ -38,22 +38,23 @@ import java.util.Comparator; import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.PriorityQueue; /** * Vector global index reader using DiskANN. * - *

This implementation uses DiskANN for efficient approximate nearest neighbor search. It - * supports lazy loading of indices and optional memory-mapped file loading for better memory - * efficiency with large indices. + *

This implementation uses DiskANN for efficient approximate nearest neighbor search. Both the + * Vamana graph and full-precision vectors are read on demand from Paimon FileIO-backed storage + * (local, HDFS, S3, OSS, etc.) via {@link SeekableInputStream}, ensuring that neither is loaded + * into Java memory in full. */ public class DiskAnnVectorGlobalIndexReader implements GlobalIndexReader { /** * Loaded search handles. Each entry wraps a DiskANN {@link IndexSearcher} (Rust native beam - * search with graph in memory, full vectors read on-demand via {@link FileIOVectorReader}). + * search with both graph and vectors read on-demand from FileIO-backed storage via {@link + * FileIOGraphReader} and {@link FileIOVectorReader}). */ private final List handles; @@ -98,8 +99,9 @@ void search( } /** - * Uses DiskANN's native Rust beam search via {@link IndexSearcher}. Graph is in memory; vectors - * are fetched on demand from object storage through {@link FileIOVectorReader} JNI callbacks. + * Uses DiskANN's native Rust beam search via {@link IndexSearcher}. Both graph and vectors are + * read on demand from Paimon FileIO-backed storage through {@link FileIOGraphReader} and {@link + * FileIOVectorReader} JNI callbacks. */ private static class DiskAnnSearchHandle implements SearchHandle { private final IndexSearcher searcher; @@ -299,8 +301,8 @@ private void ensureLoadIndices(List positions) throws IOException { /** * Load an index at the given position. * - *

The index file (header + graph) and the data file (vectors) are accessed on demand via - * {@link SeekableInputStream}s — neither is loaded into Java memory in full. The PQ pivots and + *

The index file (graph) and the data file (vectors) are accessed on demand via {@link + * SeekableInputStream}s — neither is loaded into Java memory in full. The PQ pivots and * compressed codes are loaded into memory as the "memory thumbnail" for approximate distance * computation during native beam search. */ @@ -309,29 +311,35 @@ private void loadIndexAt(int position) throws IOException { DiskAnnIndexMeta meta = indexMetas.get(position); SearchHandle handle = null; try { - // 1. Open index file (header + graph) as a SeekableInputStream. - // FileIOGraphReader scans the header + builds offset index; graph neighbors are read - // on demand during beam search. + // 1. Open index file (graph only, no header) as a SeekableInputStream. + // FileIOGraphReader scans the graph section + builds offset index; graph neighbors + // are read on demand during beam search. + // numNodes = user vectors + 1 start point. + int numNodes = (int) meta.numVectors() + 1; SeekableInputStream graphStream = fileReader.getInputStream(ioMeta); - FileIOGraphReader graphReader = new FileIOGraphReader(graphStream, VECTOR_CACHE_SIZE); + FileIOGraphReader graphReader = + new FileIOGraphReader( + graphStream, + meta.dim(), + meta.metricValue(), + meta.maxDegree(), + meta.buildListSize(), + numNodes, + meta.startId(), + VECTOR_CACHE_SIZE); // 2. Open data file stream for on-demand full-vector reads. Path dataPath = new Path(ioMeta.filePath().getParent(), meta.dataFileName()); GlobalIndexIOMeta dataIOMeta = new GlobalIndexIOMeta(dataPath, 0L, new byte[0]); SeekableInputStream vectorStream = fileReader.getInputStream(dataIOMeta); FileIOVectorReader vectorReader = - new FileIOVectorReader( - vectorStream, - graphReader.getDimension(), - 0L, // vectors start at offset 0 in the data file - buildExtIdToEntryIndex(graphReader), - VECTOR_CACHE_SIZE); + new FileIOVectorReader(vectorStream, meta.dim(), VECTOR_CACHE_SIZE); // 3. Create DiskANN native searcher with on-demand graph + vector access. handle = new DiskAnnSearchHandle( IndexSearcher.createFromReaders( - graphReader, vectorReader, graphReader.getDimension())); + graphReader, vectorReader, meta.dim(), meta.minId())); if (handles.size() <= position) { while (handles.size() < position) { @@ -347,20 +355,6 @@ private void loadIndexAt(int position) throws IOException { } } - /** - * Build the extId → entryIndex mapping from the graph reader's ID arrays. This is needed by - * {@link FileIOVectorReader} to map external vector IDs to byte offsets in the data file. - */ - private static Map buildExtIdToEntryIndex(FileIOGraphReader graphReader) { - int[] intIds = graphReader.getAllInternalIds(); - long[] extIds = graphReader.getAllExternalIds(); - Map map = new HashMap<>(intIds.length); - for (int i = 0; i < intIds.length; i++) { - map.put(extIds[i], i); - } - return map; - } - @Override public void close() throws IOException { Throwable firstException = null; diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java index a4d0a56459ef..016eed11d867 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexWriter.java @@ -33,7 +33,6 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.FloatBuffer; -import java.nio.LongBuffer; import java.util.ArrayList; import java.util.List; @@ -46,14 +45,14 @@ *

  • Vamana Graph Construction — vectors are added in batches and the Vamana graph (with * alpha-pruning) is built via the native DiskANN library. *
  • PQ Compression — after the graph is built, a Product Quantization codebook is - * trained on the vectors and all vectors are compressed to compact PQ codes. + * trained via the native DiskANN library and all vectors are compressed to compact PQ codes. * * *

    For each index flush, four files are produced: * *

      - *
    • {@code .index} — Vamana graph (header + adjacency lists) - *
    • {@code .data} — raw vector data (for exact distance reranking) + *
    • {@code .index} — Vamana graph (adjacency lists only, no header) + *
    • {@code .data} — raw vectors stored sequentially (position = ID) *
    • {@code .pq_pivots} — PQ codebook (centroids) *
    • {@code .pq_compressed} — PQ compressed codes (memory thumbnail) *
    @@ -84,9 +83,6 @@ public class DiskAnnVectorGlobalIndexWriter implements GlobalIndexSingletonWrite private DiskAnnIndex currentIndex; private boolean built = false; - /** All vectors accumulated for the current index, kept for PQ training after graph build. */ - private final List currentIndexVectors; - public DiskAnnVectorGlobalIndexWriter( GlobalIndexFileWriter fileWriter, DataType fieldType, @@ -99,7 +95,6 @@ public DiskAnnVectorGlobalIndexWriter( this.dim = options.dimension(); this.pendingBatch = new ArrayList<>(batchSize); this.results = new ArrayList<>(); - this.currentIndexVectors = new ArrayList<>(); validateFieldType(fieldType); } @@ -172,9 +167,7 @@ private void addBatchToIndex() throws IOException { int n = pendingBatch.size(); ByteBuffer vectorBuffer = DiskAnnIndex.allocateVectorBuffer(n, dim); - ByteBuffer idBuffer = DiskAnnIndex.allocateIdBuffer(n); FloatBuffer floatView = vectorBuffer.asFloatBuffer(); - LongBuffer longView = idBuffer.asLongBuffer(); for (int i = 0; i < n; i++) { VectorEntry entry = pendingBatch.get(i); @@ -182,12 +175,9 @@ private void addBatchToIndex() throws IOException { for (int j = 0; j < dim; j++) { floatView.put(i * dim + j, vector[j]); } - longView.put(i, entry.id); - // Accumulate vectors for PQ training. - currentIndexVectors.add(vector); } - currentIndex.addWithIds(vectorBuffer, idBuffer, n); + currentIndex.add(vectorBuffer, n); currentIndexCount += n; pendingBatch.clear(); built = false; @@ -204,7 +194,8 @@ private void flushCurrentIndex() throws IOException { built = true; } - // Serialize the full index (header + graph + vectors) into one buffer. + // Serialize the graph + vectors into one buffer (no header — metadata goes to + // DiskAnnIndexMeta). long serializeSize = currentIndex.serializeSize(); if (serializeSize > Integer.MAX_VALUE) { throw new IOException( @@ -221,9 +212,9 @@ private void flushCurrentIndex() throws IOException { serializeBuffer.rewind(); serializeBuffer.get(fullData); - // Parse layout to find the boundary between graph and vector sections. - FileIOVectorReader.IndexLayout layout = FileIOVectorReader.parseIndexLayout(fullData); - int vectorOffset = (int) layout.vectorSectionOffset(); + // Compute split point: data section = user vectors only (no start point). + int dataSectionSize = (int) (currentIndexCount * dim * Float.BYTES); + int graphSectionSize = fullData.length - dataSectionSize; // Generate file names — all share the same base name. String indexFileName = fileWriter.newFileName(DiskAnnVectorGlobalIndexerFactory.IDENTIFIER); @@ -232,56 +223,54 @@ private void flushCurrentIndex() throws IOException { String pqPivotsFileName = baseName + ".pq_pivots"; String pqCompressedFileName = baseName + ".pq_compressed"; - // Write index file: header + graph section only. + // Write index file: graph section only (no header). try (OutputStream out = new BufferedOutputStream(fileWriter.newOutputStream(indexFileName))) { - out.write(fullData, 0, vectorOffset); + out.write(fullData, 0, graphSectionSize); out.flush(); } - // Write data file: raw vector section only. + // Write data file: raw vectors in sequential order (position = ID). try (OutputStream out = new BufferedOutputStream(fileWriter.newOutputStream(dataFileName))) { - out.write(fullData, vectorOffset, fullData.length - vectorOffset); + out.write(fullData, graphSectionSize, dataSectionSize); out.flush(); } - // ---- Phase 1: PQ Compression & Training ---- - // Train PQ codebook on the accumulated vectors and compress them. - float[][] vectors = currentIndexVectors.toArray(new float[0][]); - ProductQuantizer pq = - ProductQuantizer.train( - vectors, - dim, + // ---- Phase 1: PQ Compression & Training (native) ---- + // Train PQ codebook on the vectors stored in the native index and encode all vectors. + byte[][] pqResult = + currentIndex.pqTrainAndEncode( options.pqSubspaces(), options.pqSampleSize(), options.pqKmeansIterations()); - byte[][] codes = pq.encodeAll(vectors); - // Write PQ pivots file (codebook). try (OutputStream out = new BufferedOutputStream(fileWriter.newOutputStream(pqPivotsFileName))) { - out.write(pq.serializePivots()); + out.write(pqResult[0]); out.flush(); } // Write PQ compressed file (memory thumbnail). try (OutputStream out = new BufferedOutputStream(fileWriter.newOutputStream(pqCompressedFileName))) { - out.write(ProductQuantizer.serializeCompressed(codes, options.pqSubspaces())); + out.write(pqResult[1]); out.flush(); } - // Build metadata with all companion file names. + // Build metadata with all companion file names and graph parameters. DiskAnnIndexMeta meta = new DiskAnnIndexMeta( dim, options.metric().toMetricType().value(), - options.indexType().value(), + 0, currentIndexCount, currentIndexMinId, currentIndexMaxId, + options.maxDegree(), + options.buildListSize(), + 0, // startId is always 0 (START_POINT_ID) dataFileName, pqPivotsFileName, pqCompressedFileName); @@ -292,7 +281,6 @@ private void flushCurrentIndex() throws IOException { currentIndexCount = 0; currentIndexMinId = Long.MAX_VALUE; currentIndexMaxId = Long.MIN_VALUE; - currentIndexVectors.clear(); built = false; } @@ -300,7 +288,6 @@ private DiskAnnIndex createIndex() { return DiskAnnIndex.create( options.dimension(), options.metric(), - options.indexType(), options.maxDegree(), options.buildListSize()); } @@ -327,7 +314,6 @@ public void close() { currentIndex.close(); currentIndex = null; } - currentIndexVectors.clear(); } /** Entry holding a vector and its row ID. */ diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorIndexOptions.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorIndexOptions.java index 8dab2a44784f..373f087cee5a 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorIndexOptions.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorIndexOptions.java @@ -38,12 +38,6 @@ public class DiskAnnVectorIndexOptions { .withDescription( "The similarity metric for vector search (L2, INNER_PRODUCT, COSINE), and L2 is the default"); - public static final ConfigOption VECTOR_INDEX_TYPE = - ConfigOptions.key("vector.diskann.index-type") - .enumType(DiskAnnIndexType.class) - .defaultValue(DiskAnnIndexType.MEMORY) - .withDescription("The DiskANN index type to use (MEMORY, DISK)"); - public static final ConfigOption VECTOR_MAX_DEGREE = ConfigOptions.key("vector.diskann.max-degree") .intType() @@ -99,7 +93,6 @@ public class DiskAnnVectorIndexOptions { private final int dimension; private final DiskAnnVectorMetric metric; - private final DiskAnnIndexType indexType; private final int maxDegree; private final int buildListSize; private final int searchListSize; @@ -112,7 +105,6 @@ public class DiskAnnVectorIndexOptions { public DiskAnnVectorIndexOptions(Options options) { this.dimension = options.get(VECTOR_DIM); this.metric = options.get(VECTOR_METRIC); - this.indexType = options.get(VECTOR_INDEX_TYPE); this.maxDegree = options.get(VECTOR_MAX_DEGREE); this.buildListSize = options.get(VECTOR_BUILD_LIST_SIZE); this.searchListSize = options.get(VECTOR_SEARCH_LIST_SIZE); @@ -123,8 +115,7 @@ public DiskAnnVectorIndexOptions(Options options) { this.searchFactor = options.get(VECTOR_SEARCH_FACTOR); int rawPqSub = options.get(VECTOR_PQ_SUBSPACES); - this.pqSubspaces = - rawPqSub > 0 ? rawPqSub : ProductQuantizer.defaultNumSubspaces(dimension); + this.pqSubspaces = rawPqSub > 0 ? rawPqSub : defaultNumSubspaces(dimension); this.pqKmeansIterations = options.get(VECTOR_PQ_KMEANS_ITERATIONS); this.pqSampleSize = options.get(VECTOR_PQ_SAMPLE_SIZE); } @@ -137,10 +128,6 @@ public DiskAnnVectorMetric metric() { return metric; } - public DiskAnnIndexType indexType() { - return indexType; - } - public int maxDegree() { return maxDegree; } @@ -175,4 +162,16 @@ public int pqKmeansIterations() { public int pqSampleSize() { return pqSampleSize; } + + /** + * Compute a reasonable default number of PQ subspaces for the given dimension. The result is + * the largest divisor of {@code dim} that is {@code <= dim / 4} and at least 1. + */ + static int defaultNumSubspaces(int dim) { + int target = Math.max(1, dim / 4); + while (target > 1 && dim % target != 0) { + target--; + } + return target; + } } diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOGraphReader.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOGraphReader.java index c366e184b106..4fbe9d964d72 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOGraphReader.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOGraphReader.java @@ -41,36 +41,34 @@ * beam search. It also calls getter methods ({@link #getDimension()}, {@link #getCount()}, etc.) * during searcher initialization. * - *

    Index file layout

    + *

    Index file layout (graph only, no header)

    * *
    - *   Header (36 bytes): 9 × i32
    - *     magic, version, dimension, metricType, indexType,
    - *     maxDegree, buildListSize, count, startId
    - *
      *   Graph section: for each node (count nodes):
    - *     ext_id       : i64
      *     int_id       : i32
      *     neighbor_cnt : i32
      *     neighbors    : neighbor_cnt × i32
      * 
    * - *

    On construction, the header is read and the graph section is scanned once sequentially to - * build an offset index (mapping internal node ID → file byte offset for its neighbor data). After - * that, individual neighbor lists are read on demand by seeking to the stored offset. + *

    DiskANN stores vectors sequentially — the position IS the ID. Internal IDs map to positions + * via {@code position = int_id - 1} for user vectors. The start point ({@code int_id == startId}) + * is not a user vector. + * + *

    All metadata (dimension, metric, max_degree, etc.) is provided externally via {@link + * DiskAnnIndexMeta} — the file contains only graph data. + * + *

    On construction, the graph section is scanned once sequentially to build an offset index + * (mapping internal node ID → file byte offset for its neighbor data). After that, individual + * neighbor lists are read on demand by seeking to the stored offset. */ public class FileIOGraphReader implements Closeable { - /** Header size: 9 × i32 = 36 bytes. */ - private static final int HEADER_SIZE = 36; - /** Source stream — must support seek(). */ private final SeekableInputStream input; - // ---- Header fields ---- + // ---- Metadata fields (from DiskAnnIndexMeta) ---- private final int dimension; private final int metricValue; - private final int indexTypeValue; private final int maxDegree; private final int buildListSize; private final int count; @@ -81,77 +79,68 @@ public class FileIOGraphReader implements Closeable { /** Mapping from internal node ID → byte offset of the node's neighbor_cnt field in the file. */ private final Map nodeNeighborOffsets; - /** All internal node IDs (in file order). */ - private final int[] allInternalIds; - - /** Corresponding external IDs (same order as {@link #allInternalIds}). */ - private final long[] allExternalIds; - /** LRU cache: internal node ID → neighbor list (int[]). */ private final LinkedHashMap cache; /** - * Create a reader by parsing the header and scanning the graph section to build the offset - * index. + * Create a reader from metadata and a seekable input stream. + * + *

    The stream should point to a file that contains ONLY the graph section (no header, no + * IDs). All metadata is supplied via parameters (originally from {@link DiskAnnIndexMeta}). * - * @param input seekable input stream for the index file (header + graph) + * @param input seekable input stream for the index file (graph only) + * @param dimension vector dimension + * @param metricValue metric type value (0=L2, 1=IP, 2=Cosine) + * @param maxDegree maximum adjacency list size + * @param buildListSize search list size used during construction + * @param count total number of graph nodes (including start point) + * @param startId internal ID of the graph start/entry point * @param cacheSize maximum number of cached neighbor lists (0 uses default 4096) * @throws IOException if reading or parsing fails */ - public FileIOGraphReader(SeekableInputStream input, int cacheSize) throws IOException { + public FileIOGraphReader( + SeekableInputStream input, + int dimension, + int metricValue, + int maxDegree, + int buildListSize, + int count, + int startId, + int cacheSize) + throws IOException { this.input = input; - - // 1. Read header. - byte[] headerBuf = new byte[HEADER_SIZE]; - input.seek(0); - readFully(input, headerBuf); - - int off = 0; - // magic(4) + version(4) — skip validation here; Rust validates during search. - off += 8; - this.dimension = readInt(headerBuf, off); - off += 4; - this.metricValue = readInt(headerBuf, off); - off += 4; - this.indexTypeValue = readInt(headerBuf, off); - off += 4; - this.maxDegree = readInt(headerBuf, off); - off += 4; - this.buildListSize = readInt(headerBuf, off); - off += 4; - this.count = readInt(headerBuf, off); - off += 4; - this.startId = readInt(headerBuf, off); - - // 2. Scan graph section to build offset index. + this.dimension = dimension; + this.metricValue = metricValue; + this.maxDegree = maxDegree; + this.buildListSize = buildListSize; + this.count = count; + this.startId = startId; + + // Scan graph section to build offset index. + // The file starts directly with graph entries (no header). + // Each entry: int_id(4) + neighbor_cnt(4) + neighbors(cnt*4). this.nodeNeighborOffsets = new HashMap<>(count); - this.allInternalIds = new int[count]; - this.allExternalIds = new long[count]; - // Reusable buffer for reading ext_id(8) + int_id(4) + neighbor_cnt(4) = 16 bytes per node. - byte[] nodeBuf = new byte[16]; - long filePos = HEADER_SIZE; + // Reusable buffer for reading int_id(4) + neighbor_cnt(4) = 8 bytes per node. + byte[] nodeBuf = new byte[8]; + long filePos = 0; for (int i = 0; i < count; i++) { input.seek(filePos); readFully(input, nodeBuf); - long extId = readLong(nodeBuf, 0); - int intId = readInt(nodeBuf, 8); - int neighborCount = readInt(nodeBuf, 12); - - allInternalIds[i] = intId; - allExternalIds[i] = extId; + int intId = readInt(nodeBuf, 0); + int neighborCount = readInt(nodeBuf, 4); // Store file offset pointing to the neighbor_cnt field (so readNeighbors can re-read // count + data). - nodeNeighborOffsets.put(intId, filePos + 12); + nodeNeighborOffsets.put(intId, filePos + 4); - // Advance past: ext_id(8) + int_id(4) + neighbor_cnt(4) + neighbors(cnt*4). - filePos += 16 + (long) neighborCount * 4; + // Advance past: int_id(4) + neighbor_cnt(4) + neighbors(cnt*4). + filePos += 8 + (long) neighborCount * 4; } - // 3. Create LRU cache. + // Create LRU cache. final int cap = cacheSize > 0 ? cacheSize : 4096; this.cache = new LinkedHashMap(cap, 0.75f, true) { @@ -172,10 +161,6 @@ public int getMetricValue() { return metricValue; } - public int getIndexTypeValue() { - return indexTypeValue; - } - public int getMaxDegree() { return maxDegree; } @@ -192,16 +177,6 @@ public int getStartId() { return startId; } - /** Return all internal node IDs (in file order). */ - public int[] getAllInternalIds() { - return allInternalIds; - } - - /** Return all external node IDs (same order as {@link #getAllInternalIds()}). */ - public long[] getAllExternalIds() { - return allExternalIds; - } - // ---- On-demand neighbor reading (called by Rust JNI during beam search) ---- /** @@ -279,15 +254,4 @@ private static int readInt(byte[] buf, int off) { | ((buf[off + 2] & 0xFF) << 16) | ((buf[off + 3] & 0xFF) << 24); } - - private static long readLong(byte[] buf, int off) { - return (buf[off] & 0xFFL) - | ((buf[off + 1] & 0xFFL) << 8) - | ((buf[off + 2] & 0xFFL) << 16) - | ((buf[off + 3] & 0xFFL) << 24) - | ((buf[off + 4] & 0xFFL) << 32) - | ((buf[off + 5] & 0xFFL) << 40) - | ((buf[off + 6] & 0xFFL) << 48) - | ((buf[off + 7] & 0xFFL) << 56); - } } diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOVectorReader.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOVectorReader.java index 528724cdfb80..079717d63a2b 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOVectorReader.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOVectorReader.java @@ -31,18 +31,21 @@ /** * Fetches vectors from a DiskANN data file through a Paimon {@link SeekableInputStream}. * - *

    The underlying stream can be backed by any Paimon FileIO provider — local, HDFS, S3, OSS (via - * Jindo SDK), etc. This class adds an LRU cache so that repeated reads for the same vector (common - * during DiskANN's beam search) do not trigger redundant I/O. + *

    The underlying stream can be backed by any Paimon FileIO provider — local, HDFS, S3, OSS, etc. + * This class adds an LRU cache so that repeated reads for the same vector (common during DiskANN's + * beam search) do not trigger redundant I/O. * *

    The Rust JNI layer invokes {@link #readVector(long)} via reflection during DiskANN's native * beam search — no specific Java interface is required. * - *

    File layout (vector section)

    + *

    Data file layout

    * - *

    Vectors are stored contiguously. Each vector occupies {@code dimension * 4} bytes - * (native-order floats). The position of a vector with entry-index {@code i} is: {@code - * vectorSectionOffset + i * dimension * 4}. + *

    Vectors are stored contiguously in sequential order. Each vector occupies {@code dimension * + * 4} bytes (native-order floats). The vector at position {@code i} is at byte offset {@code i * + * dimension * 4}. The sequential position IS the ID. + * + *

    The start point vector is NOT stored in the data file; it is handled in memory by the Rust + * native layer. */ public class FileIOVectorReader implements Closeable { @@ -52,16 +55,7 @@ public class FileIOVectorReader implements Closeable { /** Vector dimension. */ private final int dimension; - /** Byte offset of the vector section within the file. Vectors start after the graph section. */ - private final long vectorSectionOffset; - - /** - * Mapping from external (user-facing) vector ID to entry index in the file. Entry index - * determines the byte offset: {@code vectorSectionOffset + entryIndex * dimension * 4}. - */ - private final Map extIdToEntryIndex; - - /** LRU cache: external vector ID → float[]. */ + /** LRU cache: position → float[]. */ private final LinkedHashMap cache; /** Reusable byte buffer for reading a single vector. */ @@ -72,20 +66,11 @@ public class FileIOVectorReader implements Closeable { * * @param input seekable input stream for the data file * @param dimension vector dimension - * @param vectorSectionOffset byte offset where the vector section starts in the file - * @param extIdToEntryIndex mapping from external vector ID to sequential entry index * @param cacheSize maximum number of cached vectors (0 disables caching) */ - public FileIOVectorReader( - SeekableInputStream input, - int dimension, - long vectorSectionOffset, - Map extIdToEntryIndex, - int cacheSize) { + public FileIOVectorReader(SeekableInputStream input, int dimension, int cacheSize) { this.input = input; this.dimension = dimension; - this.vectorSectionOffset = vectorSectionOffset; - this.extIdToEntryIndex = extIdToEntryIndex; this.readBuf = new byte[dimension * Float.BYTES]; final int cap = Math.max(cacheSize, 16); @@ -104,39 +89,40 @@ protected boolean removeEldestEntry(Map.Entry eldest) { *

    Called by the Rust JNI layer during DiskANN's native beam search. Returns a defensive * copy — callers may freely modify the returned array without corrupting the cache. * - * @param vectorId the external (user-facing) vector ID - * @return the float vector (a fresh copy), or {@code null} if unavailable + *

    The byte offset is computed as {@code position * dimension * Float.BYTES}. + * + * @param position the 0-based position in the data file (int_id - 1 for user vectors) + * @return the float vector (a fresh copy), or {@code null} if position is negative */ - public float[] readVector(long vectorId) { + public float[] readVector(long position) { + // Start point (position = -1) is not in the data file. + if (position < 0) { + return null; + } + // 1. LRU cache hit — return a defensive copy. - float[] cached = cache.get(vectorId); + float[] cached = cache.get(position); if (cached != null) { return Arrays.copyOf(cached, cached.length); } - // 2. Look up entry index. - Integer entryIndex = extIdToEntryIndex.get(vectorId); - if (entryIndex == null) { - return null; // unknown vector - } - - // 3. Seek & read from the underlying stream. - long byteOffset = vectorSectionOffset + (long) entryIndex * dimension * Float.BYTES; + // 2. Compute byte offset: sequential position. + long byteOffset = position * dimension * Float.BYTES; try { input.seek(byteOffset); readFully(input, readBuf); } catch (IOException e) { throw new RuntimeException( - "Failed to read vector " + vectorId + " at offset " + byteOffset, e); + "Failed to read vector at position " + position + " offset " + byteOffset, e); } - // 4. Decode floats. + // 3. Decode floats. float[] vector = new float[dimension]; ByteBuffer bb = ByteBuffer.wrap(readBuf).order(ByteOrder.nativeOrder()); bb.asFloatBuffer().get(vector); - // 5. Store a separate copy in the cache so the returned array is independent. - cache.put(vectorId, Arrays.copyOf(vector, vector.length)); + // 4. Store a separate copy in the cache so the returned array is independent. + cache.put(position, Arrays.copyOf(vector, vector.length)); return vector; } @@ -161,63 +147,4 @@ private static void readFully(SeekableInputStream in, byte[] buf) throws IOExcep off += n; } } - - // ------------------------------------------------------------------ - // Factory helper — parse header to obtain vectorSectionOffset - // and extIdToEntryIndex. - // ------------------------------------------------------------------ - - /** - * Parse a serialized byte array (header + graph section) to extract the mapping and vector - * section offset needed by this reader. - * - *

    This is typically called once when opening the index for search. The byte array only needs - * to contain the header and graph section — the vector section is not accessed. - * - * @param data byte array containing at least the header and graph section - * @return a {@link IndexLayout} containing the parsed metadata - */ - public static IndexLayout parseIndexLayout(byte[] data) { - // Header: 9 × i32 = 36 bytes. - int off = 0; - // skip magic(4), version(4), dimension(4), metricType(4), indexType(4), - // maxDegree(4), buildListSize(4) - off += 28; - int count = readInt(data, off); - off += 4; - // skip startId(4) - off += 4; - - // Scan graph section to find the vector section offset. - for (int i = 0; i < count; i++) { - // skip ext_id(8) + int_id(4) - off += 12; - int neighborCount = readInt(data, off); - off += 4; - // skip neighbor IDs - off += neighborCount * 4; - } - - return new IndexLayout(off); - } - - /** Parsed layout metadata. */ - public static class IndexLayout { - private final long vectorSectionOffset; - - IndexLayout(long vectorSectionOffset) { - this.vectorSectionOffset = vectorSectionOffset; - } - - public long vectorSectionOffset() { - return vectorSectionOffset; - } - } - - private static int readInt(byte[] buf, int off) { - return (buf[off] & 0xFF) - | ((buf[off + 1] & 0xFF) << 8) - | ((buf[off + 2] & 0xFF) << 16) - | ((buf[off + 3] & 0xFF) << 24); - } } diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/ProductQuantizer.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/ProductQuantizer.java deleted file mode 100644 index 3a26d0c121e8..000000000000 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/ProductQuantizer.java +++ /dev/null @@ -1,396 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.diskann.index; - -import java.io.ByteArrayOutputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.util.Arrays; -import java.util.Random; - -/** - * Product Quantization (PQ) implementation for DiskANN. - * - *

    PQ compresses high-dimensional vectors into compact codes by: - * - *

      - *
    1. Splitting each vector into {@code M} sub-vectors (subspaces). - *
    2. Training {@code K} centroids per subspace using K-Means clustering. - *
    3. Encoding each sub-vector as the index of its nearest centroid (1 byte for K=256). - *
    - * - *

    This produces a "memory thumbnail" — e.g., a 128-dim float vector (512 bytes) is compressed to - * just 16 bytes (with M=16). During search, the compressed codes stay resident in memory and allow - * fast approximate distance computation, reducing unnecessary disk I/O for full vectors. - * - *

    File layout

    - * - *

    Pivots file ({@code .pq_pivots}): contains the trained codebook. - * - *

    - *   int: dimension (D)
    - *   int: numSubspaces (M)
    - *   int: numCentroids (K)
    - *   int: subDimension (D/M)
    - *   float[M * K * subDim]: centroid data (row-major)
    - * 
    - * - *

    Compressed file ({@code .pq_compressed}): contains the PQ codes. - * - *

    - *   int: numVectors (N)
    - *   int: numSubspaces (M)
    - *   byte[N * M]: PQ codes (row-major, one row per vector)
    - * 
    - */ -public class ProductQuantizer { - - /** Number of centroids per subspace. Fixed at 256 so each code fits in one byte (uint8). */ - public static final int NUM_CENTROIDS = 256; - - /** Default number of K-Means iterations. */ - private static final int DEFAULT_KMEANS_ITERATIONS = 20; - - /** Default maximum number of training samples. */ - private static final int DEFAULT_MAX_TRAINING_SAMPLES = 100_000; - - private final int dimension; - private final int numSubspaces; - private final int subDimension; - - /** - * Codebook: {@code centroids[m][k][d]} is the d-th component of the k-th centroid in the m-th - * subspace. - */ - private final float[][][] centroids; - - private ProductQuantizer(int dimension, int numSubspaces, float[][][] centroids) { - this.dimension = dimension; - this.numSubspaces = numSubspaces; - this.subDimension = dimension / numSubspaces; - this.centroids = centroids; - } - - // ------------------------------------------------------------------ - // Training - // ------------------------------------------------------------------ - - /** - * Train a PQ codebook from the given vectors. - * - *

    If there are more vectors than {@code maxTrainingSamples}, a random subset is used for - * training. All K-Means runs use K-Means++ initialization. - * - * @param vectors training data; each entry is a float[dimension] - * @param dimension vector dimension (must be divisible by {@code numSubspaces}) - * @param numSubspaces number of subspaces M - * @param maxTrainingSamples maximum number of samples for K-Means training - * @param kmeansIterations number of Lloyd iterations - * @return a trained {@link ProductQuantizer} - */ - public static ProductQuantizer train( - float[][] vectors, - int dimension, - int numSubspaces, - int maxTrainingSamples, - int kmeansIterations) { - if (dimension % numSubspaces != 0) { - throw new IllegalArgumentException( - "Dimension (" - + dimension - + ") must be divisible by numSubspaces (" - + numSubspaces - + ")"); - } - int subDim = dimension / numSubspaces; - int k = Math.min(NUM_CENTROIDS, vectors.length); - Random rng = new Random(42); - - // Subsample if needed. - float[][] samples = vectors; - if (vectors.length > maxTrainingSamples) { - samples = sample(vectors, maxTrainingSamples, rng); - } - - float[][][] codebook = new float[numSubspaces][k][subDim]; - for (int m = 0; m < numSubspaces; m++) { - // Extract sub-vectors for this subspace. - float[][] subVectors = extractSubspace(samples, m, subDim); - codebook[m] = kMeans(subVectors, k, kmeansIterations, rng); - } - return new ProductQuantizer(dimension, numSubspaces, codebook); - } - - // ------------------------------------------------------------------ - // Encoding - // ------------------------------------------------------------------ - - /** - * Encode a single vector into PQ codes. - * - * @param vector float[dimension] - * @return byte[numSubspaces] — one code per subspace - */ - public byte[] encode(float[] vector) { - byte[] codes = new byte[numSubspaces]; - for (int m = 0; m < numSubspaces; m++) { - int offset = m * subDimension; - float bestDist = Float.MAX_VALUE; - int bestIdx = 0; - for (int k = 0; k < centroids[m].length; k++) { - float dist = l2Squared(vector, offset, centroids[m][k], 0, subDimension); - if (dist < bestDist) { - bestDist = dist; - bestIdx = k; - } - } - codes[m] = (byte) bestIdx; - } - return codes; - } - - /** - * Encode all vectors. - * - * @return byte[numVectors][numSubspaces] - */ - public byte[][] encodeAll(float[][] vectors) { - byte[][] codes = new byte[vectors.length][]; - for (int i = 0; i < vectors.length; i++) { - codes[i] = encode(vectors[i]); - } - return codes; - } - - // ------------------------------------------------------------------ - // Serialization — Pivots - // ------------------------------------------------------------------ - - /** Serialize the codebook (pivots) to a byte array. */ - public byte[] serializePivots() throws IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream out = new DataOutputStream(baos); - out.writeInt(dimension); - out.writeInt(numSubspaces); - out.writeInt(centroids[0].length); // K - out.writeInt(subDimension); - for (int m = 0; m < numSubspaces; m++) { - for (int k = 0; k < centroids[m].length; k++) { - for (int d = 0; d < subDimension; d++) { - out.writeFloat(centroids[m][k][d]); - } - } - } - out.flush(); - return baos.toByteArray(); - } - - // ------------------------------------------------------------------ - // Serialization — Compressed codes - // ------------------------------------------------------------------ - - /** - * Serialize PQ compressed codes to a byte array. - * - * @param codes byte[numVectors][numSubspaces] - * @param numSubspaces M - * @return serialized bytes - */ - public static byte[] serializeCompressed(byte[][] codes, int numSubspaces) throws IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream out = new DataOutputStream(baos); - out.writeInt(codes.length); - out.writeInt(numSubspaces); - for (byte[] code : codes) { - out.write(code); - } - out.flush(); - return baos.toByteArray(); - } - - /** - * Compute a reasonable default number of subspaces for the given dimension. The result is the - * largest divisor of {@code dim} that is {@code <= dim / 4} and at least 1. - */ - public static int defaultNumSubspaces(int dim) { - int target = Math.max(1, dim / 4); - while (target > 1 && dim % target != 0) { - target--; - } - return target; - } - - // ------------------------------------------------------------------ - // K-Means (K-Means++ init + Lloyd iterations) - // ------------------------------------------------------------------ - - /** - * Run K-Means clustering on the given data. - * - * @param data float[n][subDim] - * @param k number of clusters - * @param maxIter maximum iterations - * @param rng random source - * @return float[k][subDim] — cluster centroids - */ - static float[][] kMeans(float[][] data, int k, int maxIter, Random rng) { - int n = data.length; - int d = data[0].length; - - if (n <= k) { - // Fewer data points than clusters — each point is its own centroid. - float[][] centroids = new float[k][d]; - for (int i = 0; i < n; i++) { - centroids[i] = Arrays.copyOf(data[i], d); - } - for (int i = n; i < k; i++) { - centroids[i] = Arrays.copyOf(data[rng.nextInt(n)], d); - } - return centroids; - } - - // K-Means++ initialization. - float[][] centroids = kMeansPPInit(data, k, rng); - - int[] assignments = new int[n]; - int[] counts = new int[k]; - - for (int iter = 0; iter < maxIter; iter++) { - // Assignment step. - for (int i = 0; i < n; i++) { - float bestDist = Float.MAX_VALUE; - int bestC = 0; - for (int c = 0; c < k; c++) { - float dist = l2Squared(data[i], 0, centroids[c], 0, d); - if (dist < bestDist) { - bestDist = dist; - bestC = c; - } - } - assignments[i] = bestC; - } - - // Update step. - float[][] newCentroids = new float[k][d]; - Arrays.fill(counts, 0); - for (int i = 0; i < n; i++) { - int c = assignments[i]; - counts[c]++; - for (int j = 0; j < d; j++) { - newCentroids[c][j] += data[i][j]; - } - } - for (int c = 0; c < k; c++) { - if (counts[c] > 0) { - for (int j = 0; j < d; j++) { - newCentroids[c][j] /= counts[c]; - } - } else { - // Re-seed empty cluster from a random data point. - newCentroids[c] = Arrays.copyOf(data[rng.nextInt(n)], d); - } - } - centroids = newCentroids; - } - return centroids; - } - - /** K-Means++ initialization: pick k initial centroids with probability proportional to D^2. */ - private static float[][] kMeansPPInit(float[][] data, int k, Random rng) { - int n = data.length; - int d = data[0].length; - float[][] centroids = new float[k][d]; - - // Pick first centroid uniformly at random. - centroids[0] = Arrays.copyOf(data[rng.nextInt(n)], d); - - float[] minDist = new float[n]; - Arrays.fill(minDist, Float.MAX_VALUE); - - for (int c = 1; c < k; c++) { - // Update minimum distances to chosen centroids. - float totalDist = 0f; - for (int i = 0; i < n; i++) { - float dist = l2Squared(data[i], 0, centroids[c - 1], 0, d); - if (dist < minDist[i]) { - minDist[i] = dist; - } - totalDist += minDist[i]; - } - - // Sample next centroid proportional to D^2. - float threshold = rng.nextFloat() * totalDist; - float cumulative = 0f; - int chosen = n - 1; - for (int i = 0; i < n; i++) { - cumulative += minDist[i]; - if (cumulative >= threshold) { - chosen = i; - break; - } - } - centroids[c] = Arrays.copyOf(data[chosen], d); - } - return centroids; - } - - // ------------------------------------------------------------------ - // Helpers - // ------------------------------------------------------------------ - - /** Compute L2 squared distance between two sub-arrays. */ - private static float l2Squared(float[] a, int offsetA, float[] b, int offsetB, int length) { - float sum = 0f; - for (int i = 0; i < length; i++) { - float diff = a[offsetA + i] - b[offsetB + i]; - sum += diff * diff; - } - return sum; - } - - /** Random sampling without replacement (Fisher-Yates on indices). */ - private static float[][] sample(float[][] data, int sampleSize, Random rng) { - int n = data.length; - int[] indices = new int[n]; - for (int i = 0; i < n; i++) { - indices[i] = i; - } - for (int i = 0; i < sampleSize; i++) { - int j = i + rng.nextInt(n - i); - int tmp = indices[i]; - indices[i] = indices[j]; - indices[j] = tmp; - } - float[][] result = new float[sampleSize][]; - for (int i = 0; i < sampleSize; i++) { - result[i] = data[indices[i]]; - } - return result; - } - - /** Extract the m-th subspace from a set of vectors. */ - private static float[][] extractSubspace(float[][] vectors, int m, int subDim) { - int offset = m * subDim; - float[][] sub = new float[vectors.length][subDim]; - for (int i = 0; i < vectors.length; i++) { - System.arraycopy(vectors[i], offset, sub[i], 0, subDim); - } - return sub; - } -} diff --git a/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexScanTest.java b/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexScanTest.java index d2e2df289127..043115e4bcb5 100644 --- a/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexScanTest.java +++ b/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexScanTest.java @@ -105,7 +105,6 @@ public void before() throws Exception { .option(CoreOptions.BUCKET.key(), "-1") .option("vector.dim", "2") .option("vector.metric", "L2") - .option("vector.diskann.index-type", "MEMORY") .option("data-evolution.enabled", "true") .option("row-tracking.enabled", "true") .build(); @@ -160,7 +159,6 @@ public void testVectorIndexScanWithDifferentMetrics() throws Exception { .option(CoreOptions.BUCKET.key(), "-1") .option("vector.dim", "2") .option("vector.metric", "INNER_PRODUCT") - .option("vector.diskann.index-type", "MEMORY") .option("data-evolution.enabled", "true") .option("row-tracking.enabled", "true") .build(); diff --git a/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexTest.java b/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexTest.java index 831877101db6..3a9d66c72a08 100644 --- a/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexTest.java +++ b/paimon-diskann/paimon-diskann-index/src/test/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexTest.java @@ -153,43 +153,36 @@ public void testDifferentMetrics() throws IOException { } @Test - public void testDifferentIndexTypes() throws IOException { + public void testDefaultOptions() throws IOException { int dimension = 32; int numVectors = 100; - String[] indexTypes = {"MEMORY"}; - - for (String indexType : indexTypes) { - Options options = createDefaultOptions(dimension); - options.setString("vector.diskann.index-type", indexType); - DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); - Path typeIndexPath = new Path(indexPath, indexType.toLowerCase()); - GlobalIndexFileWriter fileWriter = createFileWriter(typeIndexPath); - DiskAnnVectorGlobalIndexWriter writer = - new DiskAnnVectorGlobalIndexWriter(fileWriter, vectorType, indexOptions); + Options options = createDefaultOptions(dimension); + DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); + GlobalIndexFileWriter fileWriter = createFileWriter(indexPath); + DiskAnnVectorGlobalIndexWriter writer = + new DiskAnnVectorGlobalIndexWriter(fileWriter, vectorType, indexOptions); - List testVectors = generateRandomVectors(numVectors, dimension); - testVectors.forEach(writer::write); + List testVectors = generateRandomVectors(numVectors, dimension); + testVectors.forEach(writer::write); - List results = writer.finish(); - assertThat(results).hasSize(1); + List results = writer.finish(); + assertThat(results).hasSize(1); - ResultEntry result = results.get(0); - GlobalIndexFileReader fileReader = createFileReader(typeIndexPath); - List metas = new ArrayList<>(); - metas.add( - new GlobalIndexIOMeta( - new Path(typeIndexPath, result.fileName()), - fileIO.getFileSize(new Path(typeIndexPath, result.fileName())), - result.meta())); + ResultEntry result = results.get(0); + GlobalIndexFileReader fileReader = createFileReader(indexPath); + List metas = new ArrayList<>(); + metas.add( + new GlobalIndexIOMeta( + new Path(indexPath, result.fileName()), + fileIO.getFileSize(new Path(indexPath, result.fileName())), + result.meta())); - try (DiskAnnVectorGlobalIndexReader reader = - new DiskAnnVectorGlobalIndexReader( - fileReader, metas, vectorType, indexOptions)) { - VectorSearch vectorSearch = new VectorSearch(testVectors.get(0), 5, fieldName); - GlobalIndexResult searchResult = reader.visitVectorSearch(vectorSearch).get(); - assertThat(searchResult).isNotNull(); - } + try (DiskAnnVectorGlobalIndexReader reader = + new DiskAnnVectorGlobalIndexReader(fileReader, metas, vectorType, indexOptions)) { + VectorSearch vectorSearch = new VectorSearch(testVectors.get(0), 5, fieldName); + GlobalIndexResult searchResult = reader.visitVectorSearch(vectorSearch).get(); + assertThat(searchResult).isNotNull(); } } @@ -444,11 +437,94 @@ public void testBatchWriteWithRemainder() throws IOException { } } + @Test + public void testPqFilesProducedWithCorrectStructure() throws IOException { + int dimension = 8; + int numVectors = 50; + + Options options = createDefaultOptions(dimension); + DiskAnnVectorIndexOptions indexOptions = new DiskAnnVectorIndexOptions(options); + GlobalIndexFileWriter fileWriter = createFileWriter(indexPath); + DiskAnnVectorGlobalIndexWriter writer = + new DiskAnnVectorGlobalIndexWriter(fileWriter, vectorType, indexOptions); + + List testVectors = generateRandomVectors(numVectors, dimension); + testVectors.forEach(writer::write); + + List results = writer.finish(); + assertThat(results).hasSize(1); + + ResultEntry result = results.get(0); + DiskAnnIndexMeta meta = DiskAnnIndexMeta.deserialize(result.meta()); + + // Verify all four files exist. + Path indexFilePath = new Path(indexPath, result.fileName()); + Path dataFilePath = new Path(indexPath, meta.dataFileName()); + Path pqPivotsPath = new Path(indexPath, meta.pqPivotsFileName()); + Path pqCompressedPath = new Path(indexPath, meta.pqCompressedFileName()); + + assertThat(fileIO.exists(indexFilePath)).as("Index file should exist").isTrue(); + assertThat(fileIO.exists(dataFilePath)).as("Data file should exist").isTrue(); + assertThat(fileIO.exists(pqPivotsPath)).as("PQ pivots file should exist").isTrue(); + assertThat(fileIO.exists(pqCompressedPath)).as("PQ compressed file should exist").isTrue(); + + // Verify PQ pivots header: dim(i32), M(i32), K(i32), subDim(i32) + byte[] pivotsData = readAllBytesFromFile(pqPivotsPath); + assertThat(pivotsData.length).as("PQ pivots should have data").isGreaterThan(16); + + java.nio.ByteBuffer pivotsBuf = + java.nio.ByteBuffer.wrap(pivotsData).order(java.nio.ByteOrder.nativeOrder()); + int readDim = pivotsBuf.getInt(); + int readM = pivotsBuf.getInt(); + int readK = pivotsBuf.getInt(); + int readSubDim = pivotsBuf.getInt(); + + assertThat(readDim).as("PQ pivots dimension").isEqualTo(dimension); + int expectedM = indexOptions.pqSubspaces(); + assertThat(readM).as("PQ pivots num_subspaces").isEqualTo(expectedM); + assertThat(readK).as("PQ pivots num_centroids").isGreaterThan(0).isLessThanOrEqualTo(256); + assertThat(readSubDim).as("PQ pivots sub_dimension").isEqualTo(dimension / expectedM); + + // Verify total pivots file size: 16 header + M * K * subDim * 4 + int expectedPivotsSize = 16 + readM * readK * readSubDim * 4; + assertThat(pivotsData.length).as("PQ pivots file size").isEqualTo(expectedPivotsSize); + + // Verify PQ compressed header: N(i32), M(i32), then N*M bytes of codes + byte[] compressedData = readAllBytesFromFile(pqCompressedPath); + assertThat(compressedData.length).as("PQ compressed should have data").isGreaterThan(8); + + java.nio.ByteBuffer compBuf = + java.nio.ByteBuffer.wrap(compressedData).order(java.nio.ByteOrder.nativeOrder()); + int readN = compBuf.getInt(); + int readCompM = compBuf.getInt(); + + assertThat(readN).as("PQ compressed num_vectors").isEqualTo(numVectors); + assertThat(readCompM).as("PQ compressed num_subspaces").isEqualTo(expectedM); + + // Verify total compressed file size: 8 header + N * M + int expectedCompSize = 8 + readN * readCompM; + assertThat(compressedData.length).as("PQ compressed file size").isEqualTo(expectedCompSize); + + // Verify search still works with these PQ files. + GlobalIndexFileReader fileReader = createFileReader(indexPath); + List metas = new ArrayList<>(); + metas.add( + new GlobalIndexIOMeta( + indexFilePath, fileIO.getFileSize(indexFilePath), result.meta())); + + try (DiskAnnVectorGlobalIndexReader reader = + new DiskAnnVectorGlobalIndexReader(fileReader, metas, vectorType, indexOptions)) { + VectorSearch vectorSearch = new VectorSearch(testVectors.get(0), 3, fieldName); + GlobalIndexResult searchResult = reader.visitVectorSearch(vectorSearch).get(); + assertThat(searchResult).isNotNull(); + assertThat(searchResult.results().getLongCardinality()).isGreaterThan(0); + } + } + private Options createDefaultOptions(int dimension) { Options options = new Options(); options.setInteger("vector.dim", dimension); options.setString("vector.metric", "L2"); - options.setString("vector.diskann.index-type", "MEMORY"); options.setInteger("vector.diskann.max-degree", 64); options.setInteger("vector.diskann.build-list-size", 100); options.setInteger("vector.diskann.search-list-size", 100); @@ -478,6 +554,22 @@ private List generateRandomVectors(int count, int dimension) { return vectors; } + private byte[] readAllBytesFromFile(Path path) throws IOException { + int fileSize = (int) fileIO.getFileSize(path); + byte[] data = new byte[fileSize]; + try (java.io.InputStream in = fileIO.newInputStream(path)) { + int offset = 0; + while (offset < fileSize) { + int read = in.read(data, offset, fileSize - offset); + if (read < 0) { + break; + } + offset += read; + } + } + return data; + } + private boolean containsRowId(GlobalIndexResult result, long rowId) { List resultIds = new ArrayList<>(); result.results().iterator().forEachRemaining(resultIds::add); diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java index 296f288343a8..861dec172e87 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java @@ -50,9 +50,8 @@ static native long indexCreate( /** Get the metric type of an index. */ static native int indexGetMetricType(long handle); - /** Add vectors with IDs to an index using direct ByteBuffers (zero-copy). */ - static native void indexAddWithIds( - long handle, long n, ByteBuffer vectorBuffer, ByteBuffer idBuffer); + /** Add vectors to an index using a direct ByteBuffer (zero-copy). */ + static native void indexAdd(long handle, long n, ByteBuffer vectorBuffer); /** Build the index graph after adding vectors. */ static native void indexBuild(long handle, int buildListSize); @@ -92,21 +91,6 @@ static native void indexSearch( /** Return the number of bytes needed for serialization. */ static native long indexSerializeSize(long handle); - /** - * Create a search-only index from serialized data and a Java callback for on-demand - * vector reads. - * - *

    The graph adjacency lists are loaded into memory from {@code data}. During search, the - * Rust code invokes {@code vectorReader.readVector(long)} via JNI whenever a full vector is - * needed - enabling the classic DiskANN architecture where vectors live on remote object - * storage (e.g., via Jindo SDK). - * - * @param data byte[] containing serialized index data. - * @param vectorReader a Java object with a {@code float[] readVector(long)} method. - * @return a searcher handle (>= 100 000) for use with {@link #indexSearchWithReader}. - */ - static native long indexCreateSearcher(byte[] data, Object vectorReader); - /** * Create a search-only index from two on-demand readers: one for the graph structure * and one for vectors. @@ -116,20 +100,20 @@ static native void indexSearch( *

      *
    • Graph: the Rust side calls {@code graphReader.readNeighbors(int)} via JNI to * fetch neighbor lists on demand during beam search. It also calls getter methods ({@code - * getDimension()}, {@code getCount()}, {@code getStartId()}, {@code getAllInternalIds()}, - * {@code getAllExternalIds()}) during initialization. - *
    • Vectors: the Rust side calls {@code vectorReader.readVector(long)} via JNI (same - * as {@link #indexCreateSearcher}). + * getDimension()}, {@code getCount()}, {@code getStartId()}) during initialization. + *
    • Vectors: the Rust side calls {@code vectorReader.readVector(long)} via JNI. *
    * * @param graphReader a Java object providing graph structure on demand. * @param vectorReader a Java object with a {@code float[] readVector(long)} method. + * @param minExtId minimum external ID for this index (for int_id → ext_id conversion). * @return a searcher handle (>= 100 000) for use with {@link #indexSearchWithReader}. */ - static native long indexCreateSearcherFromReaders(Object graphReader, Object vectorReader); + static native long indexCreateSearcherFromReaders( + Object graphReader, Object vectorReader, long minExtId); /** - * Search on a searcher handle created by {@link #indexCreateSearcher}. + * Search on a searcher handle created by {@link #indexCreateSearcherFromReaders}. * * @see #indexSearch for parameter descriptions — semantics are identical. */ @@ -144,4 +128,21 @@ static native void indexSearchWithReader( /** Destroy a searcher handle and free its resources. */ static native void indexDestroySearcher(long handle); + + /** + * Train a Product Quantization codebook on the vectors stored in the index and encode all + * vectors into compact PQ codes. + * + *

    The index must have had vectors added via {@link #indexAddWithIds} before calling this + * method. + * + * @param handle the native index handle. + * @param numSubspaces number of PQ subspaces (M). Dimension must be divisible by M. + * @param maxSamples maximum number of vectors sampled for K-Means training. + * @param kmeansIters number of K-Means iterations. + * @return {@code byte[2]} where {@code [0]} is the serialized PQ pivots (codebook) and {@code + * [1]} is the serialized compressed PQ codes. + */ + static native byte[][] pqTrainAndEncode( + long handle, int numSubspaces, int maxSamples, int kmeansIters); } diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java index 867abfbd9881..5291aa045fd7 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/Index.java @@ -57,11 +57,10 @@ public MetricType getMetricType() { return MetricType.fromValue(DiskAnnNative.indexGetMetricType(nativeHandle)); } - public void addWithIds(long n, ByteBuffer vectorBuffer, ByteBuffer idBuffer) { + public void add(long n, ByteBuffer vectorBuffer) { checkNotClosed(); validateDirectBuffer(vectorBuffer, n * dimension * Float.BYTES, "vector"); - validateDirectBuffer(idBuffer, n * Long.BYTES, "id"); - DiskAnnNative.indexAddWithIds(nativeHandle, n, vectorBuffer, idBuffer); + DiskAnnNative.indexAdd(nativeHandle, n, vectorBuffer); } public void build(int buildListSize) { @@ -103,11 +102,10 @@ public long serializeSize() { } /** - * Serialize this index with its Vamana graph adjacency lists into the given direct ByteBuffer. + * Serialize the Vamana graph adjacency lists and vectors into the given direct ByteBuffer. * - *

    The serialized data can later be loaded by {@link IndexSearcher#create(byte[], - * java.io.Closeable)} for search-only use where vectors are read on demand from an object - * store. + *

    The serialized data contains the graph section followed by the vector section, with no + * header. Metadata (dimension, metric, etc.) is stored separately in {@code DiskAnnIndexMeta}. * * @return the number of bytes written */ @@ -127,15 +125,24 @@ public static Index create( return new Index(handle, dimension); } + /** + * Train a PQ codebook on the vectors in this index and encode all vectors. + * + * @param numSubspaces number of PQ subspaces (M). + * @param maxSamples maximum training samples for K-Means. + * @param kmeansIters number of K-Means iterations. + * @return {@code byte[2]}: [0] = serialized pivots, [1] = serialized compressed codes. + */ + public byte[][] pqTrainAndEncode(int numSubspaces, int maxSamples, int kmeansIters) { + checkNotClosed(); + return DiskAnnNative.pqTrainAndEncode(nativeHandle, numSubspaces, maxSamples, kmeansIters); + } + public static ByteBuffer allocateVectorBuffer(int numVectors, int dimension) { return ByteBuffer.allocateDirect(numVectors * dimension * Float.BYTES) .order(ByteOrder.nativeOrder()); } - public static ByteBuffer allocateIdBuffer(int numIds) { - return ByteBuffer.allocateDirect(numIds * Long.BYTES).order(ByteOrder.nativeOrder()); - } - private void validateDirectBuffer(ByteBuffer buffer, long requiredBytes, String name) { if (!buffer.isDirect()) { throw new IllegalArgumentException(name + " buffer must be a direct buffer"); diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java index f818489c8fd1..4ac2f45476c5 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java @@ -21,16 +21,20 @@ import java.io.Closeable; /** - * A search-only DiskANN index whose graph lives in memory and whose vectors are fetched - * lazily from a vector reader via JNI callbacks. + * A search-only DiskANN index backed by Paimon FileIO (local, HDFS, S3, OSS, etc.). * - *

    This implements the core DiskANN architecture: the Vamana graph is loaded from serialized data - * at construction time, while full-precision vectors are read on-demand during beam search. The - * Rust JNI code invokes the reader's {@code readVector(long)} method for each candidate node whose - * vector is not yet cached. + *

    Both the Vamana graph and full-precision vectors are read on-demand from FileIO-backed storage + * during beam search. The Rust JNI code invokes Java reader callbacks: * - *

    The vector reader must be any Java object with a {@code float[] readVector(long)} method. The - * Rust JNI layer calls this method via reflection — no specific interface is required. + *

      + *
    • {@code graphReader.readNeighbors(int)} — fetches graph neighbor lists from the {@code + * .index} file via {@code SeekableInputStream}. + *
    • {@code vectorReader.readVector(long)} — fetches full-precision vectors from the {@code + * .data} file via {@code SeekableInputStream}. + *
    + * + *

    Frequently accessed data is cached (graph neighbors in a {@code DashMap}, vectors in an LRU + * cache) to reduce FileIO/JNI round-trips. * *

    Thread Safety: instances are not thread-safe. */ @@ -53,32 +57,6 @@ private IndexSearcher(long nativeHandle, int dimension, Closeable... closeables) this.closeables = closeables; } - /** - * Create a search-only index from serialized data and a vector reader callback. - * - *

    The {@code reader} must expose a {@code float[] readVector(long)} method that the Rust JNI - * layer invokes via reflection during beam search. It must also implement {@link Closeable} so - * its resources are released when this searcher is closed. - * - * @param data the serialized byte array (header + graph + vector data). - * @param reader a Java object with a {@code readVector(long)} method, also {@link Closeable}. - * @return a new IndexSearcher - * @throws DiskAnnException if deserialization fails - */ - public static IndexSearcher create(byte[] data, Closeable reader) { - long handle = DiskAnnNative.indexCreateSearcher(data, reader); - // Parse dimension from the header (bytes 8..12, little-endian). - int dim = 0; - if (data.length >= 12) { - dim = - (data[8] & 0xFF) - | ((data[9] & 0xFF) << 8) - | ((data[10] & 0xFF) << 16) - | ((data[11] & 0xFF) << 24); - } - return new IndexSearcher(handle, dim, reader); - } - /** * Create a search-only index from two on-demand readers. * @@ -90,17 +68,18 @@ public static IndexSearcher create(byte[] data, Closeable reader) { *

  • {@code vectorReader.readVector(long)} for full-precision vectors during beam search * * - *

    Initialization reads header info and ID mappings from the graph reader (via getter - * methods). Both readers must implement {@link Closeable}. + *

    Both readers must implement {@link Closeable}. * * @param graphReader a graph reader object (e.g. {@code FileIOGraphReader}). * @param vectorReader a vector reader object (e.g. {@code FileIOVectorReader}). - * @param dimension the vector dimension (obtained from graph reader header). + * @param dimension the vector dimension (from {@code DiskAnnIndexMeta}). + * @param minExtId minimum external ID for this index (for int_id → ext_id conversion). * @return a new IndexSearcher */ public static IndexSearcher createFromReaders( - Closeable graphReader, Closeable vectorReader, int dimension) { - long handle = DiskAnnNative.indexCreateSearcherFromReaders(graphReader, vectorReader); + Closeable graphReader, Closeable vectorReader, int dimension, long minExtId) { + long handle = + DiskAnnNative.indexCreateSearcherFromReaders(graphReader, vectorReader, minExtId); return new IndexSearcher(handle, dimension, graphReader, vectorReader); } diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.lock b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.lock index 7089eddcf145..2730f7b3d5db 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.lock +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.lock @@ -86,6 +86,25 @@ dependencies = [ "memchr", ] +[[package]] +name = "crossbeam-deque" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9dd111b7b7f7d55b72c0a6ae361660ee5853c9af73f70c3c2ef6858b950e2e51" +dependencies = [ + "crossbeam-epoch", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-epoch" +version = "0.9.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b82ac4a3c2ca9c3460964f020e1402edd5753411d7737aa39c3714ad1b5420e" +dependencies = [ + "crossbeam-utils", +] + [[package]] name = "crossbeam-utils" version = "0.8.21" @@ -134,6 +153,23 @@ dependencies = [ "tracing", ] +[[package]] +name = "diskann-quantization" +version = "0.45.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f6703f9a574be9bf6c9f1f59033e127f5300cd240e50fc40dffe9a64475f92d5" +dependencies = [ + "bytemuck", + "cfg-if", + "diskann-utils", + "diskann-vector", + "diskann-wide", + "half", + "rand", + "rayon", + "thiserror 2.0.18", +] + [[package]] name = "diskann-utils" version = "0.45.0" @@ -145,6 +181,7 @@ dependencies = [ "diskann-wide", "half", "rand", + "rayon", "thiserror 2.0.18", ] @@ -169,6 +206,12 @@ dependencies = [ "half", ] +[[package]] +name = "either" +version = "1.15.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48c757948c5ede0e46177b7add2e67155f70e33c07fea8284df6576da70b3719" + [[package]] name = "foldhash" version = "0.2.0" @@ -331,6 +374,8 @@ version = "0.1.0" dependencies = [ "dashmap", "diskann", + "diskann-quantization", + "diskann-utils", "diskann-vector", "futures-util", "jni", @@ -434,6 +479,26 @@ dependencies = [ "rand", ] +[[package]] +name = "rayon" +version = "1.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "368f01d005bf8fd9b1206fb6fa653e6c4a81ceb1466406b81792d87c5677a58f" +dependencies = [ + "either", + "rayon-core", +] + +[[package]] +name = "rayon-core" +version = "1.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22e18b0f0062d30d4230b2e85ff77fdfe4326feb054b9783a3460d8435c8ab91" +dependencies = [ + "crossbeam-deque", + "crossbeam-utils", +] + [[package]] name = "redox_syscall" version = "0.5.18" diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml index 2cf6a39547c5..538d90bb393f 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml @@ -35,3 +35,5 @@ tokio = { version = "1", features = ["rt"] } futures-util = "0.3" # Concurrent HashMap used by the JniProvider graph storage dashmap = "6" +diskann-quantization = "0.45.0" +diskann-utils = "0.45.0" diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs index 21d12def0732..0a25552ed0d7 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs @@ -30,7 +30,7 @@ //! undefined behaviour and likely crash the JVM. On panic the function throws a //! `java.lang.RuntimeException` with the panic message and returns a safe default. -use jni::objects::{JByteArray, JByteBuffer, JClass, JObject, JPrimitiveArray, ReleaseMode}; +use jni::objects::{JByteBuffer, JClass, JObject, JPrimitiveArray, ReleaseMode}; use jni::sys::{jfloat, jint, jlong}; use jni::JNIEnv; @@ -43,8 +43,9 @@ use diskann::graph::{self, DiskANNIndex}; use diskann::neighbor::{BackInserter, Neighbor}; use diskann_vector::distance::Metric; -mod jni_provider; -use jni_provider::{JniContext, JniProvider, JniStrategy}; +mod paimon_fileio_provider; +mod pq; +use paimon_fileio_provider::{FileIOContext, FileIOProvider, FileIOStrategy}; // ======================== Constants ======================== @@ -52,11 +53,6 @@ const METRIC_L2: i32 = 0; const METRIC_INNER_PRODUCT: i32 = 1; const METRIC_COSINE: i32 = 2; -/// Serialization magic number ("PDAN"). -const MAGIC: i32 = 0x5044414E; -/// Serialization format version (3 = graph + vectors). -const SERIALIZE_VERSION: i32 = 3; - /// The u32 ID reserved for the DiskANN graph start/entry point. const START_POINT_ID: u32 = 0; @@ -103,15 +99,11 @@ struct IndexState { dimension: i32, metric_type: i32, - index_type: i32, - max_degree: usize, - build_list_size: usize, - ext_to_int: HashMap, - int_to_ext: HashMap, next_id: u32, - raw_data: Vec<(i64, Vec)>, + /// Vectors stored in insertion order. Position i has int_id = i + 1. + raw_data: Vec>, } // ======================== Registry ======================== @@ -152,7 +144,7 @@ fn get_index(handle: i64) -> Option>> { fn create_index_state( dimension: i32, metric_type: i32, - index_type: i32, + _index_type: i32, max_degree: i32, build_list_size: i32, ) -> Result { @@ -193,11 +185,6 @@ fn create_index_state( runtime, dimension, metric_type, - index_type, - max_degree: md, - build_list_size: bls, - ext_to_int: HashMap::new(), - int_to_ext: HashMap::new(), next_id: START_POINT_ID + 1, raw_data: Vec::new(), }) @@ -220,22 +207,6 @@ fn get_direct_buffer_slice<'a>( // ======================== Serialization Helpers ======================== -fn read_i32(buf: &[u8], offset: &mut usize) -> Option { - if *offset + 4 > buf.len() { return None; } - let mut b = [0u8; 4]; - b.copy_from_slice(&buf[*offset..*offset + 4]); - *offset += 4; - Some(i32::from_ne_bytes(b)) -} - -fn read_i64(buf: &[u8], offset: &mut usize) -> Option { - if *offset + 8 > buf.len() { return None; } - let mut b = [0u8; 8]; - b.copy_from_slice(&buf[*offset..*offset + 8]); - *offset += 8; - Some(i64::from_ne_bytes(b)) -} - fn write_i32(buf: &mut [u8], offset: &mut usize, v: i32) -> bool { if *offset + 4 > buf.len() { return false; } buf[*offset..*offset + 4].copy_from_slice(&v.to_ne_bytes()); @@ -243,13 +214,6 @@ fn write_i32(buf: &mut [u8], offset: &mut usize, v: i32) -> bool { true } -fn write_i64(buf: &mut [u8], offset: &mut usize, v: i64) -> bool { - if *offset + 8 > buf.len() { return false; } - buf[*offset..*offset + 8].copy_from_slice(&v.to_ne_bytes()); - *offset += 8; - true -} - fn write_f32(buf: &mut [u8], offset: &mut usize, v: f32) -> bool { if *offset + 4 > buf.len() { return false; } buf[*offset..*offset + 4].copy_from_slice(&v.to_ne_bytes()); @@ -338,13 +302,12 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexGetMetr } #[no_mangle] -pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexAddWithIds<'local>( +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexAdd<'local>( mut env: JNIEnv<'local>, _class: JClass<'local>, handle: jlong, n: jlong, vector_buffer: JByteBuffer<'local>, - id_buffer: JByteBuffer<'local>, ) { let arc = match get_index(handle) { Some(a) => a, @@ -364,7 +327,6 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexAddWith let num = n as usize; let dimension = state.dimension as usize; let vec_len = num * dimension * 4; - let id_len = num * 8; let vec_bytes = match get_direct_buffer_slice(&mut env, &vector_buffer, vec_len) { Some(slice) => slice, @@ -373,30 +335,19 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexAddWith return; } }; - let id_bytes = match get_direct_buffer_slice(&mut env, &id_buffer, id_len) { - Some(slice) => slice, - None => { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid id buffer"); - return; - } - }; let vectors = unsafe { std::slice::from_raw_parts(vec_bytes.as_ptr() as *const f32, num * dimension) }; - let ids = unsafe { std::slice::from_raw_parts(id_bytes.as_ptr() as *const i64, num) }; let strat = test_provider::Strategy::new(); for i in 0..num { - let ext_id = ids[i]; let base = i * dimension; let vector = vectors[base..base + dimension].to_vec(); let int_id = state.next_id; state.next_id += 1; - state.ext_to_int.insert(ext_id, int_id); - state.int_to_ext.insert(int_id, ext_id); - state.raw_data.push((ext_id, vector.clone())); + state.raw_data.push(vector.clone()); // catch_unwind around the DiskANN graph insert which may panic. let idx_clone = Arc::clone(&state.index); @@ -410,14 +361,14 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexAddWith Ok(Err(e)) => { let _ = env.throw_new( "java/lang/RuntimeException", - format!("DiskANN insert failed for id {}: {}", ext_id, e), + format!("DiskANN insert failed for int_id {}: {}", int_id, e), ); return; } Err(_) => { let _ = env.throw_new( "java/lang/RuntimeException", - format!("DiskANN insert panicked for id {}", ext_id), + format!("DiskANN insert panicked for int_id {}", int_id), ); return; } @@ -556,8 +507,7 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearch< continue; } let idx = qi * top_k + count; - result_labels[idx] = - *state.int_to_ext.get(&neighbor.id).unwrap_or(&(neighbor.id as i64)); + result_labels[idx] = (neighbor.id as i64) - 1; result_distances[idx] = neighbor.distance; count += 1; } @@ -599,34 +549,41 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearch< } // ============================================================================ -// Serialization format (graph + vectors) +// Serialization format (graph + data, no header) // ============================================================================ // -// Layout: -// Header : 9 × i32 (magic, version=3, dimension, metric, index_type, -// max_degree, build_list_size, count, start_id) -// Graph : for each node (ordered by ascending internal id): -// ext_id : i64 +// The index file (.index) contains ONLY the graph adjacency lists: +// Graph : for each node (start point + user vectors): // int_id : i32 // neighbor_cnt : i32 // neighbors : neighbor_cnt × i32 -// Vectors : for each node (same order): +// +// The data file (.data) contains ONLY raw vectors stored sequentially: +// Data : for each user vector (in order 0, 1, 2, ...): // vector : dim × f32 // -// During search-only mode the Rust side loads the graph into a `JniProvider` -// and invokes the Java `FileIOVectorReader.readVector(long)` callback -// for every vector access that is NOT a start point. +// The sequential position IS the ID. +// The start point is NOT stored in the data file. +// position = int_id - 1 for user vectors (int_id > 0). +// +// All metadata (dimension, metric, max_degree, build_list_size, count, +// start_id) is stored in DiskAnnIndexMeta — not in the file. +// +// During search, both graph and vector data are read on demand from +// Paimon FileIO-backed storage (local, HDFS, S3, OSS, etc.) via JNI callbacks: +// - Graph: FileIOGraphReader.readNeighbors(int) +// - Vectors: FileIOVectorReader.readVector(long) // ============================================================================ -// ---- Searcher registry (handles backed by JniProvider) ---- +// ---- Searcher registry (handles backed by FileIOProvider) ---- struct SearcherState { - index: Arc>, - context: JniContext, + index: Arc>, + context: FileIOContext, runtime: tokio::runtime::Runtime, dimension: i32, - /// Mapping from internal DiskANN u32 IDs to external Java long IDs. - int_to_ext: HashMap, + /// Minimum external ID for this index. ext_id = min_ext_id + (int_id - 1). + min_ext_id: i64, } struct SearcherRegistry { @@ -679,33 +636,16 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSeriali let provider = state.index.provider(); let dim = state.dimension as usize; + let num_user_vectors = state.raw_data.len(); + let num_nodes = num_user_vectors + 1; // +1 for start point - // Build ordered list of (int_id, ext_id, neighbors) using the async + // Build ordered list of (int_id, neighbors) using the async // NeighborAccessor API run synchronously on our tokio runtime. - // Include the start point (int_id=0) so the search can navigate from it. + // Node order: start point (int_id=0) first, then user vectors (int_id=1,2,...). let mut graph_section_size: usize = 0; - let mut graph_entries: Vec<(u32, i64, Vec)> = Vec::new(); + let mut graph_entries: Vec<(u32, Vec)> = Vec::with_capacity(num_nodes); - // Start point first (ext_id = -1 as sentinel — never a real user ID). - { - use diskann::graph::AdjacencyList; - use diskann::provider::{DefaultAccessor, NeighborAccessor as NeighborAccessorTrait}; - let accessor = provider.default_accessor(); - let mut adj = AdjacencyList::::new(); - let mut neighbors = Vec::new(); - if state.runtime.block_on(accessor.get_neighbors(START_POINT_ID, &mut adj)).is_ok() { - neighbors = adj.iter().copied().collect(); - } - graph_section_size += 8 + 4 + 4 + neighbors.len() * 4; - graph_entries.push((START_POINT_ID, -1i64, neighbors)); - } - - // User-added vectors. - for (ext_id, _vec) in &state.raw_data { - let int_id = match state.ext_to_int.get(ext_id) { - Some(i) => *i, - None => continue, - }; + for int_id in 0..num_nodes as u32 { let mut neighbors = Vec::new(); { use diskann::graph::AdjacencyList; @@ -717,15 +657,13 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSeriali neighbors = adj.iter().copied().collect(); } } - - graph_section_size += 8 + 4 + 4 + neighbors.len() * 4; // ext_id + int_id + cnt + neighbors - graph_entries.push((int_id, *ext_id, neighbors)); + graph_section_size += 4 + 4 + neighbors.len() * 4; // int_id + cnt + neighbors + graph_entries.push((int_id, neighbors)); } - let count = graph_entries.len(); // includes start point - let vector_section_size = count * dim * 4; - let header_size = 9 * 4; // 9 i32s - let total_size = header_size + graph_section_size + vector_section_size; + // Data section: user vectors in sequential order (no start point). + let data_section_size = num_user_vectors * dim * 4; + let total_size = graph_section_size + data_section_size; let buf = match get_direct_buffer_slice(&mut env, &buffer, total_size) { Some(s) => s, @@ -733,20 +671,9 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSeriali }; let mut off = 0usize; - // Header - write_i32(buf, &mut off, MAGIC); - write_i32(buf, &mut off, SERIALIZE_VERSION); - write_i32(buf, &mut off, state.dimension); - write_i32(buf, &mut off, state.metric_type); - write_i32(buf, &mut off, state.index_type); - write_i32(buf, &mut off, state.max_degree as i32); - write_i32(buf, &mut off, state.build_list_size as i32); - write_i32(buf, &mut off, count as i32); - write_i32(buf, &mut off, START_POINT_ID as i32); - - // Graph section - for (int_id, ext_id, neighbors) in &graph_entries { - write_i64(buf, &mut off, *ext_id); + + // Graph section: int_id(i32) + neighbor_cnt(i32) + neighbors(cnt × i32) + for (int_id, neighbors) in &graph_entries { write_i32(buf, &mut off, *int_id as i32); write_i32(buf, &mut off, neighbors.len() as i32); for &n in neighbors { @@ -754,22 +681,10 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSeriali } } - // Vector section (same order as graph_entries) - for (int_id, ext_id, _) in &graph_entries { - if *int_id == START_POINT_ID { - // Write the start point's dummy vector. - for _ in 0..dim { - write_f32(buf, &mut off, 1.0); - } - } else if let Some((_, vec)) = state.raw_data.iter().find(|(id, _)| id == ext_id) { - for &v in vec { - write_f32(buf, &mut off, v); - } - } else { - // Zero-fill for missing vectors. - for _ in 0..dim { - write_f32(buf, &mut off, 0.0); - } + // Data section: user vectors in insertion order. + for vec in &state.raw_data { + for &v in vec { + write_f32(buf, &mut off, v); } } @@ -794,28 +709,12 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSeriali let dim = state.dimension as usize; - // Calculate size by iterating over all graph nodes (start point + raw_data). + // Calculate size by iterating over all graph nodes (start point + user vectors). let provider = state.index.provider(); + let num_nodes = state.raw_data.len() + 1; // +1 for start point let mut graph_section_size: usize = 0; - // Start point. - { - use diskann::graph::AdjacencyList; - use diskann::provider::{DefaultAccessor, NeighborAccessor as NeighborAccessorTrait}; - let accessor = provider.default_accessor(); - let mut adj = AdjacencyList::::new(); - let nc = if state.runtime.block_on(accessor.get_neighbors(START_POINT_ID, &mut adj)).is_ok() { - adj.len() - } else { 0 }; - graph_section_size += 8 + 4 + 4 + nc * 4; - } - - // User-added vectors. - for (ext_id, _) in &state.raw_data { - let int_id = match state.ext_to_int.get(ext_id) { - Some(i) => *i, - None => continue, - }; + for int_id in 0..num_nodes as u32 { let neighbor_count = { use diskann::graph::AdjacencyList; use diskann::provider::{DefaultAccessor, NeighborAccessor as NeighborAccessorTrait}; @@ -827,201 +726,12 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSeriali 0 } }; - graph_section_size += 8 + 4 + 4 + neighbor_count * 4; + graph_section_size += 4 + 4 + neighbor_count * 4; // int_id + cnt + neighbors } - let count = state.raw_data.len() + 1; // +1 for start point - let header_size = 9 * 4; - let vector_section_size = count * dim * 4; - (header_size + graph_section_size + vector_section_size) as jlong -} - -// ======================== indexCreateSearcher ======================== - -/// Create a search-only handle from serialized data + Java callback reader. -/// -/// `data`: byte[] containing the serialized index. -/// `vectorReader`: Java object with a `readVector(long)` method (e.g. `FileIOVectorReader`). -/// -/// Returns a searcher handle (≥100000) that can be used with `indexSearchWithReader`. -#[no_mangle] -pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateSearcher<'local>( - mut env: JNIEnv<'local>, - _class: JClass<'local>, - data: JByteArray<'local>, - vector_reader: JObject<'local>, -) -> jlong { - let bytes = match env.convert_byte_array(&data) { - Ok(d) => d, - Err(_) => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid data"); return 0; } - }; - - let mut off = 0usize; - - // Parse header. - let magic = read_i32(&bytes, &mut off).unwrap_or(0); - if magic != MAGIC { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid magic"); - return 0; - } - let version = read_i32(&bytes, &mut off).unwrap_or(0); - if version != SERIALIZE_VERSION { - let _ = env.throw_new("java/lang/IllegalArgumentException", - format!("Expected version {}, got {}", SERIALIZE_VERSION, version)); - return 0; - } - let dimension = read_i32(&bytes, &mut off).unwrap_or(0); - let metric_type = read_i32(&bytes, &mut off).unwrap_or(0); - let _index_type = read_i32(&bytes, &mut off).unwrap_or(0); - let max_degree = read_i32(&bytes, &mut off).unwrap_or(64) as usize; - let build_ls = read_i32(&bytes, &mut off).unwrap_or(100) as usize; - let count = read_i32(&bytes, &mut off).unwrap_or(0) as usize; - let start_id = read_i32(&bytes, &mut off).unwrap_or(0) as u32; - let dim = dimension as usize; - - // Parse graph section. - let mut graph_data: Vec<(u32, i64, Vec)> = Vec::with_capacity(count); - let mut int_to_ext: HashMap = HashMap::with_capacity(count); - - for _ in 0..count { - let ext_id = match read_i64(&bytes, &mut off) { - Some(v) => v, - None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated graph"); return 0; } - }; - let int_id = match read_i32(&bytes, &mut off) { - Some(v) => v as u32, - None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated graph"); return 0; } - }; - let ncnt = match read_i32(&bytes, &mut off) { - Some(v) => v as usize, - None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated graph"); return 0; } - }; - let mut neighbors = Vec::with_capacity(ncnt); - for _ in 0..ncnt { - let n = match read_i32(&bytes, &mut off) { - Some(v) => v as u32, - None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated graph"); return 0; } - }; - neighbors.push(n); - } - int_to_ext.insert(int_id, ext_id); - graph_data.push((int_id, ext_id, neighbors)); - } - - // Create JNI global ref for the vector reader callback (needed before - // we fetch the start-point vector below). - let global_reader = match env.new_global_ref(&vector_reader) { - Ok(g) => g, - Err(e) => { - let _ = env.throw_new("java/lang/RuntimeException", - format!("Failed to create global ref: {}", e)); - return 0; - } - }; - - let jvm = match env.get_java_vm() { - Ok(vm) => vm, - Err(e) => { - let _ = env.throw_new("java/lang/RuntimeException", - format!("Failed to get JavaVM: {}", e)); - return 0; - } - }; - - // Fetch start-point vector via JNI callback. - // The byte[] passed to this function only contains header + graph - // (no vector section), so vectors are read on-demand from the reader. - // The start point has ext_id = -1 (sentinel); its vector is the dummy - // [1.0, …] written during serialization. We try the reader first, - // falling back to a non-zero dummy to avoid division-by-zero with - // cosine metric. - let start_ext_id = graph_data.iter() - .find(|(iid, _, _)| *iid == start_id) - .map(|(_, eid, _)| *eid) - .unwrap_or(-1); - let start_vec = { - // Try reading from the Java reader callback. - let fetched = (|| -> Option> { - let mut e = jvm.attach_current_thread().ok()?; - let result = e.call_method( - &global_reader, "readVector", "(J)[F", - &[jni::objects::JValue::Long(start_ext_id)], - ).ok()?; - let arr = result.l().ok()?; - if arr.is_null() { return None; } - let jarr: jni::objects::JFloatArray = arr.into(); - let len = e.get_array_length(&jarr).ok()? as usize; - if len == 0 { return None; } - let mut buf = vec![0.0f32; len]; - e.get_float_array_region(&jarr, 0, &mut buf).ok()?; - Some(buf) - })(); - fetched.unwrap_or_else(|| vec![1.0f32; dim]) - }; - - // Build the JniProvider (graph in memory, vectors via JNI callback). - let provider = JniProvider::new( - graph_data, - start_id, - start_vec, - jvm, - global_reader, - dim, - metric_type, - max_degree, - ); - - // Build DiskANNIndex config. - let md = std::cmp::max(max_degree, 4); - let bls = std::cmp::max(build_ls, md); - let metric = map_metric(metric_type); - - let index_config = match graph::config::Builder::new( - md, - graph::config::MaxDegree::same(), - bls, - metric.into(), - ).build() { - Ok(c) => c, - Err(e) => { - let _ = env.throw_new("java/lang/RuntimeException", - format!("Failed to create index config: {:?}", e)); - return 0; - } - }; - - let runtime = match tokio::runtime::Builder::new_current_thread() - .enable_all() - .build() - { - Ok(r) => r, - Err(e) => { - let _ = env.throw_new("java/lang/RuntimeException", - format!("Failed to create tokio runtime: {}", e)); - return 0; - } - }; - - // DiskANNIndex takes ownership of the provider. After construction, - // the provider is accessible via index.provider() for read access. - // We store int_to_ext separately for mapping search results. - let index = Arc::new(DiskANNIndex::new(index_config, provider, None)); - - let searcher = SearcherState { - index, - context: JniContext, - runtime, - dimension, - int_to_ext, - }; - - match searcher_registry().lock() { - Ok(mut guard) => guard.insert(searcher), - Err(_) => { - let _ = env.throw_new("java/lang/IllegalStateException", "Searcher registry error"); - 0 - } - } + // Data section: only user vectors (no start point). + let data_section_size = state.raw_data.len() * dim * 4; + (graph_section_size + data_section_size) as jlong } // ======================== indexCreateSearcherFromReaders ======================== @@ -1031,9 +741,9 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS /// /// `graphReader`: Java object with `readNeighbors(int)`, `getDimension()`, /// `getCount()`, `getStartId()`, `getMaxDegree()`, -/// `getBuildListSize()`, `getMetricValue()`, -/// `getAllInternalIds()`, `getAllExternalIds()`. +/// `getBuildListSize()`, `getMetricValue()`. /// `vectorReader`: Java object with `readVector(long)`. +/// `min_ext_id`: Minimum external ID for int_id → ext_id conversion. /// /// Returns a searcher handle (≥100000) for use with `indexSearchWithReader`. #[no_mangle] @@ -1042,6 +752,7 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS _class: JClass<'local>, graph_reader: JObject<'local>, vector_reader: JObject<'local>, + min_ext_id: jlong, ) -> jlong { // Helper to call int-returning methods on graphReader. macro_rules! call_int { @@ -1055,56 +766,12 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS let dimension = call_int!("getDimension"); let metric_type = call_int!("getMetricValue"); - let _index_type = call_int!("getIndexTypeValue"); let max_degree = call_int!("getMaxDegree") as usize; let build_ls = call_int!("getBuildListSize") as usize; let count = call_int!("getCount") as usize; let start_id = call_int!("getStartId") as u32; let dim = dimension as usize; - // Read int_to_ext mapping via getAllInternalIds() / getAllExternalIds(). - let int_ids: Vec = { - let result = match env.call_method(&graph_reader, "getAllInternalIds", "()[I", &[]) { - Ok(v) => v, - Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("getAllInternalIds failed: {}", e)); return 0; } - }; - let obj = match result.l() { - Ok(o) => o, - Err(_) => { let _ = env.throw_new("java/lang/RuntimeException", "Bad return from getAllInternalIds"); return 0; } - }; - let arr = jni::objects::JIntArray::from(obj); - let len = env.get_array_length(&arr).unwrap_or(0) as usize; - let mut buf = vec![0i32; len]; - let _ = env.get_int_array_region(&arr, 0, &mut buf); - buf - }; - - let ext_ids: Vec = { - let result = match env.call_method(&graph_reader, "getAllExternalIds", "()[J", &[]) { - Ok(v) => v, - Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("getAllExternalIds failed: {}", e)); return 0; } - }; - let obj = match result.l() { - Ok(o) => o, - Err(_) => { let _ = env.throw_new("java/lang/RuntimeException", "Bad return from getAllExternalIds"); return 0; } - }; - let arr = jni::objects::JLongArray::from(obj); - let len = env.get_array_length(&arr).unwrap_or(0) as usize; - let mut buf = vec![0i64; len]; - let _ = env.get_long_array_region(&arr, 0, &mut buf); - buf - }; - - // Build ID mappings. - let mut int_to_ext: HashMap = HashMap::with_capacity(count); - let mut ext_to_int: HashMap = HashMap::with_capacity(count); - for i in 0..std::cmp::min(int_ids.len(), ext_ids.len()) { - let iid = int_ids[i] as u32; - let eid = ext_ids[i]; - int_to_ext.insert(iid, eid); - ext_to_int.insert(eid, iid); - } - // Create global refs for both readers. let global_graph_reader = match env.new_global_ref(&graph_reader) { Ok(g) => g, @@ -1120,31 +787,12 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("get JVM: {}", e)); return 0; } }; - // Fetch start-point vector via vectorReader JNI callback. - let start_ext_id = int_to_ext.get(&start_id).copied().unwrap_or(-1); - let start_vec = { - let fetched = (|| -> Option> { - let mut e = jvm.attach_current_thread().ok()?; - let result = e.call_method( - &global_vector_reader, "readVector", "(J)[F", - &[jni::objects::JValue::Long(start_ext_id)], - ).ok()?; - let arr = result.l().ok()?; - if arr.is_null() { return None; } - let jarr: jni::objects::JFloatArray = arr.into(); - let len = e.get_array_length(&jarr).ok()? as usize; - if len == 0 { return None; } - let mut buf = vec![0.0f32; len]; - e.get_float_array_region(&jarr, 0, &mut buf).ok()?; - Some(buf) - })(); - fetched.unwrap_or_else(|| vec![1.0f32; dim]) - }; + // Start point is not stored in data file; use a dummy vector. + let start_vec = vec![1.0f32; dim]; - // Build the JniProvider with on-demand graph reading. - let provider = JniProvider::new_with_readers( - int_to_ext.clone(), - ext_to_int, + // Build the FileIOProvider with on-demand graph reading. + let provider = FileIOProvider::new_with_readers( + count, start_id, start_vec, jvm, @@ -1182,10 +830,10 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS let searcher = SearcherState { index, - context: JniContext, + context: FileIOContext, runtime, dimension, - int_to_ext, + min_ext_id, }; match searcher_registry().lock() { @@ -1196,7 +844,7 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS // ======================== indexSearchWithReader ======================== -/// Search on a searcher handle created by `indexCreateSearcher`. +/// Search on a searcher handle created by `indexCreateSearcherFromReaders`. #[no_mangle] pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearchWithReader<'local>( mut env: JNIEnv<'local>, @@ -1239,7 +887,7 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearchW let mut result_dist = vec![f32::MAX; total]; let mut result_lbl = vec![-1i64; total]; - let strat = JniStrategy::new(); + let strat = FileIOStrategy::new(); for qi in 0..num { let qvec = &query[qi * dimension..(qi + 1) * dimension]; @@ -1288,7 +936,7 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearchW let nb = &neighbors[ri]; if nb.id == START_POINT_ID { continue; } let idx = qi * top_k + cnt; - result_lbl[idx] = state.int_to_ext.get(&nb.id).copied().unwrap_or(nb.id as i64); + result_lbl[idx] = state.min_ext_id + (nb.id as i64) - 1; result_dist[idx] = nb.distance; cnt += 1; } @@ -1327,3 +975,133 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexDestroy } })); } + +// ======================== PQ Train & Encode ======================== + +/// Train a PQ codebook on the vectors stored in the index and encode all vectors. +/// +/// Uses `diskann-quantization`'s `LightPQTrainingParameters` for K-Means++ / Lloyd +/// training and `BasicTable` for encoding. +/// +/// Returns a `byte[][]` where: +/// `[0]` = serialized pivots (codebook) +/// `[1]` = serialized compressed codes +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_pqTrainAndEncode<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + handle: jlong, + num_subspaces: jint, + max_samples: jint, + kmeans_iters: jint, +) -> JObject<'local> { + // Obtain the index state outside of catch_unwind so we can throw typed exceptions. + let arc = match get_index(handle) { + Some(a) => a, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); + return JObject::null(); + } + }; + let state = match arc.lock() { + Ok(s) => s, + Err(_) => { + let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); + return JObject::null(); + } + }; + + let dim = state.dimension as usize; + let m = num_subspaces as usize; + let max_s = max_samples as usize; + let iters = kmeans_iters as usize; + + // Perform PQ training and encoding inside catch_unwind to prevent panics crossing FFI. + let pq_result = { + let raw_data = &state.raw_data; + let result = panic::catch_unwind(AssertUnwindSafe(|| { + pq::train_and_encode(raw_data, dim, m, max_s, iters) + })); + // Drop the lock before JNI object creation. + drop(state); + match result { + Ok(Ok(r)) => r, + Ok(Err(msg)) => { + let _ = env.throw_new("java/lang/RuntimeException", msg); + return JObject::null(); + } + Err(payload) => { + let msg = if let Some(s) = payload.downcast_ref::<&str>() { + s.to_string() + } else if let Some(s) = payload.downcast_ref::() { + s.clone() + } else { + "Unknown Rust panic in PQ training".to_string() + }; + let _ = env.throw_new("java/lang/RuntimeException", msg); + return JObject::null(); + } + } + }; + + // Build Java byte[][] result. + let pivots_array = match env.byte_array_from_slice(&pq_result.pivots_bytes) { + Ok(a) => a, + Err(e) => { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!("Failed to create pivots byte[]: {}", e), + ); + return JObject::null(); + } + }; + let compressed_array = match env.byte_array_from_slice(&pq_result.compressed_bytes) { + Ok(a) => a, + Err(e) => { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!("Failed to create compressed byte[]: {}", e), + ); + return JObject::null(); + } + }; + + let byte_array_class = match env.find_class("[B") { + Ok(c) => c, + Err(e) => { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!("Failed to find [B class: {}", e), + ); + return JObject::null(); + } + }; + + let result = match env.new_object_array(2, &byte_array_class, &JObject::null()) { + Ok(a) => a, + Err(e) => { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!("Failed to create byte[][]: {}", e), + ); + return JObject::null(); + } + }; + + if let Err(e) = env.set_object_array_element(&result, 0, &pivots_array) { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!("Failed to set pivots: {}", e), + ); + return JObject::null(); + } + if let Err(e) = env.set_object_array_element(&result, 1, &compressed_array) { + let _ = env.throw_new( + "java/lang/RuntimeException", + format!("Failed to set compressed: {}", e), + ); + return JObject::null(); + } + + result.into() +} diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/jni_provider.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs similarity index 57% rename from paimon-diskann/paimon-diskann-jni/src/main/native/src/jni_provider.rs rename to paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs index d8b4723b98cb..6db3e727f6f9 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/src/jni_provider.rs +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs @@ -17,14 +17,19 @@ * under the License. */ -//! A DiskANN [`DataProvider`] whose graph lives in memory but whose vectors -//! are fetched on demand from Java via JNI callbacks. +//! A DiskANN [`DataProvider`] backed by Paimon FileIO (local, HDFS, S3, OSS, etc.). //! -//! This enables the classic DiskANN architecture: -//! - **In-memory**: navigational graph (adjacency lists) + start-point vectors. -//! - **On-demand**: full-precision vectors are read through -//! `FileIOVectorReader.readVector(long)` on the Java side, which can -//! be backed by Paimon FileIO (local, HDFS, S3, OSS, etc.). +//! Both the navigational graph (adjacency lists) and full-precision vectors +//! are stored in Paimon FileIO-backed storage and read on demand via JNI +//! callbacks to Java reader objects: +//! +//! - **Graph**: read through `FileIOGraphReader.readNeighbors(int)`, which +//! reads from a `SeekableInputStream` over the `.index` file. +//! - **Vectors**: read through `FileIOVectorReader.readVector(long)`, which +//! reads from a `SeekableInputStream` over the `.data` file. +//! +//! Frequently accessed neighbors and vectors are cached in a `DashMap` and +//! an LRU cache respectively, to reduce FileIO/JNI round-trips. use std::collections::HashMap; @@ -44,34 +49,34 @@ use crate::map_metric; // ======================== Error ======================== #[derive(Debug, Clone)] -pub enum JniProviderError { +pub enum FileIOProviderError { InvalidId(u32), JniCallFailed(String), } -impl std::fmt::Display for JniProviderError { +impl std::fmt::Display for FileIOProviderError { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { Self::InvalidId(id) => write!(f, "invalid vector id {}", id), - Self::JniCallFailed(msg) => write!(f, "JNI callback failed: {}", msg), + Self::JniCallFailed(msg) => write!(f, "FileIO read failed: {}", msg), } } } -impl std::error::Error for JniProviderError {} +impl std::error::Error for FileIOProviderError {} -impl From for ANNError { +impl From for ANNError { #[track_caller] - fn from(e: JniProviderError) -> ANNError { + fn from(e: FileIOProviderError) -> ANNError { ANNError::opaque(e) } } -diskann::always_escalate!(JniProviderError); +diskann::always_escalate!(FileIOProviderError); // ======================== LRU Cache ======================== -/// Tiny LRU cache for recently fetched vectors to reduce JNI round-trips. +/// Tiny LRU cache for recently fetched vectors to reduce FileIO/JNI round-trips. struct VectorCache { map: HashMap>, order: Vec, @@ -108,30 +113,31 @@ impl VectorCache { // ======================== Graph Term ======================== -/// One entry in the in-memory graph: its neighbors and external ID. +/// One entry in the graph cache: its neighbor list. pub struct GraphTerm { pub neighbors: AdjacencyList, - pub ext_id: i64, } -// ======================== JniProvider ======================== +// ======================== FileIOProvider ======================== -/// Data provider that keeps the graph in-memory (or lazily loaded) and reads vectors via JNI. -pub struct JniProvider { - /// In-memory graph: internal_id → { neighbors, ext_id }. - /// In on-demand mode this acts as a lazy cache — entries are populated on first access. +/// DiskANN data provider backed by Paimon FileIO. +/// +/// Graph neighbors and vectors are read on demand from FileIO-backed storage +/// (local, HDFS, S3, OSS, etc.) via JNI callbacks to Java reader objects. +/// A `DashMap` lazily caches graph entries to reduce repeated FileIO reads. +pub struct FileIOProvider { + /// Graph cache: internal_id → { neighbors }. + /// Acts as a lazy cache — entries are populated on first access from FileIO. graph: DashMap, - /// Internal→External ID mapping (separate for fast lookup). - int_to_ext: HashMap, - /// External→Internal ID mapping. - ext_to_int: HashMap, + /// Total number of nodes (start point + user vectors). + num_nodes: usize, /// Start-point IDs and their vectors (always kept in memory). start_points: HashMap>, /// JVM handle for attaching threads. jvm: JavaVM, - /// Global reference to the Java vector reader object (e.g. `FileIOVectorReader`). + /// Global reference to the Java vector reader object (`FileIOVectorReader`). reader_ref: GlobalRef, - /// Global reference to the Java graph reader object (e.g. `FileIOGraphReader`). + /// Global reference to the Java graph reader object (`FileIOGraphReader`). /// When set, graph neighbors are fetched on demand via JNI callbacks. graph_reader_ref: Option, /// Vector dimension. @@ -142,9 +148,9 @@ pub struct JniProvider { max_degree: usize, } -impl std::fmt::Debug for JniProvider { +impl std::fmt::Debug for FileIOProvider { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("JniProvider") + f.debug_struct("FileIOProvider") .field("dim", &self.dim) .field("metric", &self.metric) .field("max_degree", &self.max_degree) @@ -154,84 +160,16 @@ impl std::fmt::Debug for JniProvider { } // SAFETY: JavaVM is Send+Sync, GlobalRef is Send+Sync. -unsafe impl Send for JniProvider {} -unsafe impl Sync for JniProvider {} - -impl JniProvider { - /// Build a new search-only provider from pre-constructed graph data. - /// - /// * `graph_data`: (internal_id, external_id, neighbors) tuples. - /// * `start_id`: internal ID of the graph start point. - /// * `start_vec`: vector for the start point (kept in memory). - /// * `jvm`: Java VM reference for JNI callbacks. - /// * `reader_ref`: global ref to the Java vector reader. - /// * `dim`: vector dimension. - /// * `metric_type`:metric enum value (0=L2, 1=IP, 2=Cosine). - /// * `max_degree`: maximum adjacency list size. - pub fn new( - graph_data: Vec<(u32, i64, Vec)>, - start_id: u32, - start_vec: Vec, - jvm: JavaVM, - reader_ref: GlobalRef, - dim: usize, - metric_type: i32, - max_degree: usize, - ) -> Self { - let graph = DashMap::new(); - let mut int_to_ext = HashMap::new(); - let mut ext_to_int = HashMap::new(); - - for (int_id, ext_id, neighbors) in &graph_data { - let adj = AdjacencyList::from_iter_untrusted(neighbors.iter().copied()); - graph.insert( - *int_id, - GraphTerm { - neighbors: adj, - ext_id: *ext_id, - }, - ); - int_to_ext.insert(*int_id, *ext_id); - ext_to_int.insert(*ext_id, *int_id); - } - - // Ensure start point is in the graph too. - if !graph.contains_key(&start_id) { - graph.insert( - start_id, - GraphTerm { - neighbors: AdjacencyList::new(), - ext_id: start_id as i64, - }, - ); - int_to_ext.insert(start_id, start_id as i64); - ext_to_int.insert(start_id as i64, start_id); - } - - let mut start_points = HashMap::new(); - start_points.insert(start_id, start_vec); - - Self { - graph, - int_to_ext, - ext_to_int, - start_points, - jvm, - reader_ref, - graph_reader_ref: None, - dim, - metric: map_metric(metric_type), - max_degree, - } - } +unsafe impl Send for FileIOProvider {} +unsafe impl Sync for FileIOProvider {} +impl FileIOProvider { /// Build a search-only provider with on-demand graph reading (no pre-loaded graph data). /// /// The graph `DashMap` starts empty and acts as a lazy cache — entries are populated - /// on first access via JNI callback to `graphReader.readNeighbors(int)`. + /// on first access via FileIO through `graphReader.readNeighbors(int)`. pub fn new_with_readers( - int_to_ext: HashMap, - ext_to_int: HashMap, + num_nodes: usize, start_id: u32, start_vec: Vec, jvm: JavaVM, @@ -243,15 +181,12 @@ impl JniProvider { ) -> Self { let graph = DashMap::new(); - // The start point needs an entry so `to_internal_id` works. - // Its neighbors will be fetched on demand. let mut start_points = HashMap::new(); start_points.insert(start_id, start_vec); Self { graph, - int_to_ext, - ext_to_int, + num_nodes, start_points, jvm, reader_ref, @@ -270,9 +205,9 @@ impl JniProvider { self.metric } - /// Fetch neighbor list from Java via JNI callback to `graphReader.readNeighbors(int)`. - /// Returns None if graphReader is not set. - fn fetch_neighbors_jni(&self, int_id: u32) -> Result>, JniProviderError> { + /// Fetch neighbor list from FileIO-backed storage via JNI callback to + /// `graphReader.readNeighbors(int)`. Returns None if graphReader is not set. + fn fetch_neighbors(&self, int_id: u32) -> Result>, FileIOProviderError> { let graph_ref = match &self.graph_reader_ref { Some(r) => r, None => return Ok(None), @@ -281,7 +216,7 @@ impl JniProvider { let mut env = self .jvm .attach_current_thread() - .map_err(|e| JniProviderError::JniCallFailed(format!("attach failed: {}", e)))?; + .map_err(|e| FileIOProviderError::JniCallFailed(format!("attach failed: {}", e)))?; let result = env.call_method( graph_ref, @@ -294,7 +229,7 @@ impl JniProvider { Ok(v) => v, Err(e) => { let _ = env.exception_clear(); - return Err(JniProviderError::JniCallFailed(format!( + return Err(FileIOProviderError::JniCallFailed(format!( "readNeighbors({}) failed: {}", int_id, e ))); @@ -313,38 +248,39 @@ impl JniProvider { let int_array = jni::objects::JIntArray::from(obj); let len = env .get_array_length(&int_array) - .map_err(|e| JniProviderError::JniCallFailed(format!("get_array_length: {}", e)))? + .map_err(|e| FileIOProviderError::JniCallFailed(format!("get_array_length: {}", e)))? as usize; let mut buf = vec![0i32; len]; env.get_int_array_region(&int_array, 0, &mut buf) - .map_err(|e| JniProviderError::JniCallFailed(format!("get_int_array_region: {}", e)))?; + .map_err(|e| FileIOProviderError::JniCallFailed(format!("get_int_array_region: {}", e)))?; Ok(Some(buf.into_iter().map(|v| v as u32).collect())) } - /// Fetch a vector from Java via JNI. Returns None if readVector returns null. - fn fetch_vector_jni(&self, ext_id: i64) -> Result>, JniProviderError> { + /// Fetch a vector from FileIO-backed storage via JNI callback to + /// `vectorReader.readVector(long)`. The `position` is the 0-based index + /// in the data file (position = int_id - 1). + fn fetch_vector(&self, position: i64) -> Result>, FileIOProviderError> { let mut env = self .jvm .attach_current_thread() - .map_err(|e| JniProviderError::JniCallFailed(format!("attach failed: {}", e)))?; + .map_err(|e| FileIOProviderError::JniCallFailed(format!("attach failed: {}", e)))?; let result = env.call_method( &self.reader_ref, "readVector", "(J)[F", - &[jni::objects::JValue::Long(ext_id as jlong)], + &[jni::objects::JValue::Long(position as jlong)], ); let ret_val = match result { Ok(v) => v, Err(e) => { - // Clear any pending Java exception so the JNI env stays usable. let _ = env.exception_clear(); - return Err(JniProviderError::JniCallFailed(format!( + return Err(FileIOProviderError::JniCallFailed(format!( "readVector({}) failed: {}", - ext_id, e + position, e ))); } }; @@ -362,13 +298,13 @@ impl JniProvider { let float_array = jni::objects::JFloatArray::from(obj); let len = env .get_array_length(&float_array) - .map_err(|e| JniProviderError::JniCallFailed(format!("get_array_length: {}", e)))? + .map_err(|e| FileIOProviderError::JniCallFailed(format!("get_array_length: {}", e)))? as usize; let mut buf = vec![0f32; len]; env.get_float_array_region(&float_array, 0, &mut buf) .map_err(|e| { - JniProviderError::JniCallFailed(format!("get_float_array_region: {}", e)) + FileIOProviderError::JniCallFailed(format!("get_float_array_region: {}", e)) })?; Ok(Some(buf)) @@ -377,66 +313,65 @@ impl JniProvider { // ======================== Context ======================== -/// Lightweight execution context (mirrors test_provider::Context). +/// Lightweight execution context for the FileIO provider. #[derive(Debug, Clone, Default)] -pub struct JniContext; +pub struct FileIOContext; -impl std::fmt::Display for JniContext { +impl std::fmt::Display for FileIOContext { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "jni context") + write!(f, "paimon fileio context") } } -impl provider::ExecutionContext for JniContext {} +impl provider::ExecutionContext for FileIOContext {} // ======================== DataProvider ======================== -impl provider::DataProvider for JniProvider { - type Context = JniContext; +impl provider::DataProvider for FileIOProvider { + type Context = FileIOContext; type InternalId = u32; type ExternalId = u32; - type Error = JniProviderError; + type Error = FileIOProviderError; fn to_internal_id( &self, - _context: &JniContext, + _context: &FileIOContext, gid: &u32, - ) -> Result { - // Check ID mapping (works for both in-memory and on-demand modes). - if self.int_to_ext.contains_key(gid) || self.start_points.contains_key(gid) { + ) -> Result { + if (*gid as usize) < self.num_nodes { Ok(*gid) } else { - Err(JniProviderError::InvalidId(*gid)) + Err(FileIOProviderError::InvalidId(*gid)) } } fn to_external_id( &self, - _context: &JniContext, + _context: &FileIOContext, id: u32, - ) -> Result { - if self.int_to_ext.contains_key(&id) || self.start_points.contains_key(&id) { + ) -> Result { + if (id as usize) < self.num_nodes { Ok(id) } else { - Err(JniProviderError::InvalidId(id)) + Err(FileIOProviderError::InvalidId(id)) } } } // ======================== SetElement (stub — search only) ======================== -impl provider::SetElement<[f32]> for JniProvider { +impl provider::SetElement<[f32]> for FileIOProvider { type SetError = ANNError; type Guard = provider::NoopGuard; async fn set_element( &self, - _context: &JniContext, + _context: &FileIOContext, _id: &u32, _element: &[f32], ) -> Result { - Err(ANNError::opaque(JniProviderError::JniCallFailed( - "set_element not supported on search-only JniProvider".to_string(), + Err(ANNError::opaque(FileIOProviderError::JniCallFailed( + "set_element not supported on search-only FileIOProvider".to_string(), ))) } } @@ -444,44 +379,43 @@ impl provider::SetElement<[f32]> for JniProvider { // ======================== NeighborAccessor ======================== #[derive(Debug, Clone, Copy)] -pub struct JniNeighborAccessor<'a> { - provider: &'a JniProvider, +pub struct FileIONeighborAccessor<'a> { + provider: &'a FileIOProvider, } -impl provider::HasId for JniNeighborAccessor<'_> { +impl provider::HasId for FileIONeighborAccessor<'_> { type Id = u32; } -impl provider::NeighborAccessor for JniNeighborAccessor<'_> { +impl provider::NeighborAccessor for FileIONeighborAccessor<'_> { async fn get_neighbors( self, id: Self::Id, neighbors: &mut AdjacencyList, ) -> ANNResult { - // 1. Try in-memory graph (populated upfront or cached from previous JNI calls). + // 1. Try cached graph (populated upfront or cached from previous FileIO reads). if let Some(term) = self.provider.graph.get(&id) { neighbors.overwrite_trusted(&term.neighbors); return Ok(self); } - // 2. On-demand: fetch from Java graph reader via JNI callback. + // 2. On-demand: fetch from FileIO-backed storage via graph reader JNI callback. if self.provider.graph_reader_ref.is_some() { - let fetched = self.provider.fetch_neighbors_jni(id)?; + let fetched = self.provider.fetch_neighbors(id)?; if let Some(neighbor_ids) = fetched { let adj = AdjacencyList::from_iter_untrusted(neighbor_ids.iter().copied()); neighbors.overwrite_trusted(&adj); // Cache in the DashMap for subsequent accesses. - let ext_id = self.provider.int_to_ext.get(&id).copied().unwrap_or(id as i64); - self.provider.graph.insert(id, GraphTerm { neighbors: adj, ext_id }); + self.provider.graph.insert(id, GraphTerm { neighbors: adj }); return Ok(self); } } - Err(ANNError::opaque(JniProviderError::InvalidId(id))) + Err(ANNError::opaque(FileIOProviderError::InvalidId(id))) } } -impl provider::NeighborAccessorMut for JniNeighborAccessor<'_> { +impl provider::NeighborAccessorMut for FileIONeighborAccessor<'_> { async fn set_neighbors(self, id: Self::Id, neighbors: &[Self::Id]) -> ANNResult { match self.provider.graph.get_mut(&id) { Some(mut term) => { @@ -489,7 +423,7 @@ impl provider::NeighborAccessorMut for JniNeighborAccessor<'_> { term.neighbors.extend_from_slice(neighbors); Ok(self) } - None => Err(ANNError::opaque(JniProviderError::InvalidId(id))), + None => Err(ANNError::opaque(FileIOProviderError::InvalidId(id))), } } @@ -499,34 +433,34 @@ impl provider::NeighborAccessorMut for JniNeighborAccessor<'_> { term.neighbors.extend_from_slice(neighbors); Ok(self) } - None => Err(ANNError::opaque(JniProviderError::InvalidId(id))), + None => Err(ANNError::opaque(FileIOProviderError::InvalidId(id))), } } } // ======================== DefaultAccessor ======================== -impl provider::DefaultAccessor for JniProvider { - type Accessor<'a> = JniNeighborAccessor<'a>; +impl provider::DefaultAccessor for FileIOProvider { + type Accessor<'a> = FileIONeighborAccessor<'a>; fn default_accessor(&self) -> Self::Accessor<'_> { - JniNeighborAccessor { provider: self } + FileIONeighborAccessor { provider: self } } } // ======================== Accessor ======================== -/// Accessor that fetches vectors via JNI callback. +/// Accessor that fetches vectors from FileIO-backed storage via JNI callback. /// -/// Keeps a local buffer and an LRU cache to reduce JNI round-trips. -pub struct JniAccessor<'a> { - provider: &'a JniProvider, +/// Keeps a local buffer and an LRU cache to reduce FileIO/JNI round-trips. +pub struct FileIOAccessor<'a> { + provider: &'a FileIOProvider, buffer: Box<[f32]>, cache: VectorCache, } -impl<'a> JniAccessor<'a> { - pub fn new(provider: &'a JniProvider, cache_size: usize) -> Self { +impl<'a> FileIOAccessor<'a> { + pub fn new(provider: &'a FileIOProvider, cache_size: usize) -> Self { let buffer = vec![0.0f32; provider.dim()].into_boxed_slice(); Self { provider, @@ -536,15 +470,15 @@ impl<'a> JniAccessor<'a> { } } -impl provider::HasId for JniAccessor<'_> { +impl provider::HasId for FileIOAccessor<'_> { type Id = u32; } -impl provider::Accessor for JniAccessor<'_> { +impl provider::Accessor for FileIOAccessor<'_> { type Extended = Box<[f32]>; type Element<'a> = &'a [f32] where Self: 'a; type ElementRef<'a> = &'a [f32]; - type GetError = JniProviderError; + type GetError = FileIOProviderError; async fn get_element( &mut self, @@ -562,15 +496,11 @@ impl provider::Accessor for JniAccessor<'_> { return Ok(&*self.buffer); } - // 3. JNI callback to Java: FileIOVectorReader.readVector(extId). - let ext_id = self - .provider - .int_to_ext - .get(&id) - .copied() - .unwrap_or(id as i64); + // 3. Fetch from FileIO-backed storage via FileIOVectorReader.readVector(position). + // position = int_id - 1 (start point is int_id=0, user vectors start at 1). + let position = (id as i64) - 1; - let fetched = self.provider.fetch_vector_jni(ext_id)?; + let fetched = self.provider.fetch_vector(position)?; match fetched { Some(vec) if vec.len() == self.provider.dim() => { @@ -578,24 +508,24 @@ impl provider::Accessor for JniAccessor<'_> { self.cache.put(id, vec.into_boxed_slice()); Ok(&*self.buffer) } - Some(vec) => Err(JniProviderError::JniCallFailed(format!( + Some(vec) => Err(FileIOProviderError::JniCallFailed(format!( "readVector({}) returned {} floats, expected {}", - ext_id, + position, vec.len(), self.provider.dim() ))), - None => Err(JniProviderError::InvalidId(id)), + None => Err(FileIOProviderError::InvalidId(id)), } } } // ======================== DelegateNeighbor ======================== -impl<'this> provider::DelegateNeighbor<'this> for JniAccessor<'_> { - type Delegate = JniNeighborAccessor<'this>; +impl<'this> provider::DelegateNeighbor<'this> for FileIOAccessor<'_> { + type Delegate = FileIONeighborAccessor<'this>; fn delegate_neighbor(&'this mut self) -> Self::Delegate { - JniNeighborAccessor { + FileIONeighborAccessor { provider: self.provider, } } @@ -603,7 +533,7 @@ impl<'this> provider::DelegateNeighbor<'this> for JniAccessor<'_> { // ======================== BuildQueryComputer ======================== -impl provider::BuildQueryComputer<[f32]> for JniAccessor<'_> { +impl provider::BuildQueryComputer<[f32]> for FileIOAccessor<'_> { type QueryComputerError = diskann::error::Infallible; type QueryComputer = ::QueryDistance; @@ -620,7 +550,7 @@ impl provider::BuildQueryComputer<[f32]> for JniAccessor<'_> { // ======================== BuildDistanceComputer ======================== -impl provider::BuildDistanceComputer for JniAccessor<'_> { +impl provider::BuildDistanceComputer for FileIOAccessor<'_> { type DistanceComputerError = diskann::error::Infallible; type DistanceComputer = ::Distance; @@ -636,7 +566,7 @@ impl provider::BuildDistanceComputer for JniAccessor<'_> { // ======================== SearchExt ======================== -impl glue::SearchExt for JniAccessor<'_> { +impl glue::SearchExt for FileIOAccessor<'_> { fn starting_points( &self, ) -> impl std::future::Future>> + Send { @@ -646,34 +576,34 @@ impl glue::SearchExt for JniAccessor<'_> { // ======================== Blanket traits ======================== -impl glue::ExpandBeam<[f32]> for JniAccessor<'_> {} -impl glue::FillSet for JniAccessor<'_> {} +impl glue::ExpandBeam<[f32]> for FileIOAccessor<'_> {} +impl glue::FillSet for FileIOAccessor<'_> {} // ======================== Strategy ======================== -/// Search-only strategy for the JNI provider. +/// Search-only strategy for the Paimon FileIO provider. #[derive(Debug, Default, Clone, Copy)] -pub struct JniStrategy; +pub struct FileIOStrategy; -impl JniStrategy { +impl FileIOStrategy { pub fn new() -> Self { Self } } -impl glue::SearchStrategy for JniStrategy { +impl glue::SearchStrategy for FileIOStrategy { type QueryComputer = ::QueryDistance; type PostProcessor = glue::CopyIds; type SearchAccessorError = diskann::error::Infallible; - type SearchAccessor<'a> = JniAccessor<'a>; + type SearchAccessor<'a> = FileIOAccessor<'a>; fn search_accessor<'a>( &'a self, - provider: &'a JniProvider, - _context: &'a JniContext, - ) -> Result, diskann::error::Infallible> { - // Cache up to 1024 recently fetched vectors to reduce JNI round-trips. - Ok(JniAccessor::new(provider, 1024)) + provider: &'a FileIOProvider, + _context: &'a FileIOContext, + ) -> Result, diskann::error::Infallible> { + // Cache up to 1024 recently fetched vectors to reduce FileIO round-trips. + Ok(FileIOAccessor::new(provider, 1024)) } fn post_processor(&self) -> Self::PostProcessor { @@ -682,24 +612,24 @@ impl glue::SearchStrategy for JniStrategy { } // For insert (graph construction) — delegates to prune/search accessors. -// We implement InsertStrategy and PruneStrategy as stubs since the JniProvider +// We implement InsertStrategy and PruneStrategy as stubs since the FileIOProvider // is search-only. DiskANNIndex::new() requires the Provider to be Sized but // does NOT call insert methods unless we invoke index.insert(). -impl glue::PruneStrategy for JniStrategy { +impl glue::PruneStrategy for FileIOStrategy { type DistanceComputer = ::Distance; - type PruneAccessor<'a> = JniAccessor<'a>; + type PruneAccessor<'a> = FileIOAccessor<'a>; type PruneAccessorError = diskann::error::Infallible; fn prune_accessor<'a>( &'a self, - provider: &'a JniProvider, - _context: &'a JniContext, + provider: &'a FileIOProvider, + _context: &'a FileIOContext, ) -> Result, Self::PruneAccessorError> { - Ok(JniAccessor::new(provider, 1024)) + Ok(FileIOAccessor::new(provider, 1024)) } } -impl glue::InsertStrategy for JniStrategy { +impl glue::InsertStrategy for FileIOStrategy { type PruneStrategy = Self; fn prune_strategy(&self) -> Self::PruneStrategy { @@ -708,14 +638,14 @@ impl glue::InsertStrategy for JniStrategy { fn insert_search_accessor<'a>( &'a self, - provider: &'a JniProvider, - _context: &'a JniContext, + provider: &'a FileIOProvider, + _context: &'a FileIOContext, ) -> Result, Self::SearchAccessorError> { - Ok(JniAccessor::new(provider, 1024)) + Ok(FileIOAccessor::new(provider, 1024)) } } -impl<'a> glue::AsElement<&'a [f32]> for JniAccessor<'a> { +impl<'a> glue::AsElement<&'a [f32]> for FileIOAccessor<'a> { type Error = diskann::error::Infallible; fn as_element( &mut self, diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/pq.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/pq.rs new file mode 100644 index 000000000000..8dfb5cb7e3df --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/pq.rs @@ -0,0 +1,316 @@ +/* + * 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. + */ + +//! Product Quantization (PQ) for DiskANN, backed by `diskann-quantization`. +//! +//! PQ compresses high-dimensional vectors into compact codes by: +//! 1. Splitting each vector into `M` sub-vectors (subspaces). +//! 2. Training `K` centroids per subspace using K-Means clustering +//! (via `diskann-quantization`'s `LightPQTrainingParameters`). +//! 3. Encoding each sub-vector as the index of its nearest centroid (1 byte for K≤256). + +use diskann_quantization::cancel::DontCancel; +use diskann_quantization::product::train::{LightPQTrainingParameters, TrainQuantizer}; +use diskann_quantization::product::BasicTable; +use diskann_quantization::random::StdRngBuilder; +use diskann_quantization::views::ChunkOffsetsView; +use diskann_quantization::{CompressInto, Parallelism}; +use diskann_utils::views::Matrix; + +/// Maximum centroids per subspace. Fixed at 256 so each code fits in one byte (u8). +const NUM_CENTROIDS: usize = 256; + +/// Result of PQ training and encoding. +#[derive(Debug)] +pub struct PQResult { + /// Serialized PQ codebook (pivots). + pub pivots_bytes: Vec, + /// Serialized compressed PQ codes. + pub compressed_bytes: Vec, +} + +/// Train a PQ codebook and encode all vectors using `diskann-quantization`. +/// +/// * `vectors` — training vectors (stored sequentially, position = ID). +/// * `dimension` — vector dimension (must be divisible by `num_subspaces`). +/// * `num_subspaces` — number of PQ subspaces (M). +/// * `max_samples` — maximum number of vectors sampled for training. +/// * `kmeans_iters` — number of Lloyd iterations for K-Means. +/// +/// Returns serialized pivots and compressed codes. +pub fn train_and_encode( + vectors: &[Vec], + dimension: usize, + num_subspaces: usize, + max_samples: usize, + kmeans_iters: usize, +) -> Result { + if dimension == 0 || num_subspaces == 0 || dimension % num_subspaces != 0 { + return Err(format!( + "Invalid PQ params: dim={}, num_subspaces={}", + dimension, num_subspaces + )); + } + + let n = vectors.len(); + if n == 0 { + return Err("No vectors to train PQ".to_string()); + } + + let sub_dim = dimension / num_subspaces; + let k = std::cmp::min(NUM_CENTROIDS, n); + + // --- Build training data as a Matrix (nrows=num_samples, ncols=dim) --- + let sample_n = std::cmp::min(n, max_samples); + let sample_indices = if n > max_samples { + sample_indices_det(n, max_samples) + } else { + (0..n).collect() + }; + + let mut training_data = Matrix::new(0.0f32, sample_n, dimension); + for (dst_row, &src_idx) in sample_indices.iter().enumerate() { + training_data + .row_mut(dst_row) + .copy_from_slice(&vectors[src_idx]); + } + + // --- Build chunk offsets (uniform subspaces) --- + // e.g. for dim=8, M=2: offsets = [0, 4, 8] + let offsets: Vec = (0..=num_subspaces).map(|i| i * sub_dim).collect(); + let schema = ChunkOffsetsView::new(&offsets) + .map_err(|e| format!("Failed to create PQ chunk offsets: {}", e))?; + + // --- Train using diskann-quantization --- + let trainer = LightPQTrainingParameters::new(k, kmeans_iters); + let rng_builder = StdRngBuilder::new(42); + + let quantizer = trainer + .train( + training_data.as_view(), + schema, + Parallelism::Sequential, + &rng_builder, + &DontCancel, + ) + .map_err(|e| format!("PQ training failed: {}", e))?; + + // --- Build BasicTable for encoding --- + let ncenters = quantizer.pivots()[0].nrows(); + let flat_pivots: Vec = quantizer.flatten(); + let pivots_matrix = Matrix::try_from(flat_pivots.into_boxed_slice(), ncenters, dimension) + .map_err(|e| format!("Failed to create pivot matrix: {}", e))?; + + let offsets_owned = schema.to_owned(); + let table = BasicTable::new(pivots_matrix, offsets_owned) + .map_err(|e| format!("Failed to create BasicTable: {}", e))?; + + // --- Encode all vectors --- + let mut all_codes: Vec> = Vec::with_capacity(n); + for vec in vectors.iter() { + let mut code = vec![0u8; num_subspaces]; + table + .compress_into(vec.as_slice(), &mut code) + .map_err(|e| format!("PQ compression failed: {}", e))?; + all_codes.push(code); + } + + // --- Serialize --- + let pivots_bytes = serialize_pivots(&table, dimension, num_subspaces, ncenters, sub_dim); + let compressed_bytes = serialize_compressed(&all_codes, num_subspaces); + + Ok(PQResult { + pivots_bytes, + compressed_bytes, + }) +} + +/// Serialize the PQ codebook (pivots) to bytes. +/// +/// Format (native byte order): +/// ```text +/// i32: dimension +/// i32: num_subspaces (M) +/// i32: num_centroids (K) +/// i32: sub_dimension +/// f32[M * K * sub_dim]: centroid data (stored per-subspace) +/// ``` +fn serialize_pivots( + table: &BasicTable, + dimension: usize, + num_subspaces: usize, + num_centroids: usize, + sub_dim: usize, +) -> Vec { + // The pivots in BasicTable are stored row-major: ncenters rows × dim columns. + // Each row has all subspaces concatenated. + // We need to serialize in the per-subspace format expected by the reader: + // for subspace m: for centroid k: float[sub_dim] + let header_size = 4 * 4; + let data_size = num_subspaces * num_centroids * sub_dim * 4; + let mut buf = Vec::with_capacity(header_size + data_size); + + buf.extend_from_slice(&(dimension as i32).to_ne_bytes()); + buf.extend_from_slice(&(num_subspaces as i32).to_ne_bytes()); + buf.extend_from_slice(&(num_centroids as i32).to_ne_bytes()); + buf.extend_from_slice(&(sub_dim as i32).to_ne_bytes()); + + let pivots_view = table.view_pivots(); + // Reorder from row-major (centroid × full_dim) to subspace-major: + // subspace m, centroid k → row k, columns [m*sub_dim .. (m+1)*sub_dim] + for m in 0..num_subspaces { + let col_start = m * sub_dim; + for k in 0..num_centroids { + let row = pivots_view.row(k); + for d in 0..sub_dim { + buf.extend_from_slice(&row[col_start + d].to_ne_bytes()); + } + } + } + + buf +} + +/// Serialize compressed PQ codes to bytes. +/// +/// Format (native byte order): +/// ```text +/// i32: num_vectors (N) +/// i32: num_subspaces (M) +/// byte[N * M]: PQ codes +/// ``` +fn serialize_compressed(codes: &[Vec], num_subspaces: usize) -> Vec { + let header_size = 4 * 2; + let data_size = codes.len() * num_subspaces; + let mut buf = Vec::with_capacity(header_size + data_size); + + buf.extend_from_slice(&(codes.len() as i32).to_ne_bytes()); + buf.extend_from_slice(&(num_subspaces as i32).to_ne_bytes()); + + for code in codes { + buf.extend_from_slice(code); + } + buf +} + +/// Deterministic sampling without replacement (Fisher-Yates on indices). +fn sample_indices_det(n: usize, sample_size: usize) -> Vec { + let mut indices: Vec = (0..n).collect(); + let mut rng = SimpleRng::new(42); + + let m = std::cmp::min(sample_size, n); + for i in 0..m { + let j = i + rng.next_usize(n - i); + indices.swap(i, j); + } + indices.truncate(m); + indices +} + +/// Minimal deterministic PRNG (xorshift64). +struct SimpleRng { + state: u64, +} + +impl SimpleRng { + fn new(seed: u64) -> Self { + Self { + state: if seed == 0 { 1 } else { seed }, + } + } + + fn next_u64(&mut self) -> u64 { + let mut x = self.state; + x ^= x << 13; + x ^= x >> 7; + x ^= x << 17; + self.state = x; + x + } + + fn next_usize(&mut self, bound: usize) -> usize { + (self.next_u64() % bound as u64) as usize + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_pq_train_encode_roundtrip() { + let dim = 8; + let num_subspaces = 2; + let n = 50; + + // Generate some simple vectors. + let vectors: Vec> = (0..n) + .map(|i| { + (0..dim).map(|d| (i * dim + d) as f32 * 0.01).collect() + }) + .collect(); + + let result = train_and_encode(&vectors, dim, num_subspaces, 100, 5).unwrap(); + + // Check pivots serialization. + assert!(result.pivots_bytes.len() > 16); + + // Verify header: dim=8, M=2, K=min(256,50)=50, sub_dim=4 + let dim_read = i32::from_ne_bytes(result.pivots_bytes[0..4].try_into().unwrap()); + let m_read = i32::from_ne_bytes(result.pivots_bytes[4..8].try_into().unwrap()); + let k_read = i32::from_ne_bytes(result.pivots_bytes[8..12].try_into().unwrap()); + let sub_dim_read = i32::from_ne_bytes(result.pivots_bytes[12..16].try_into().unwrap()); + assert_eq!(dim_read, 8); + assert_eq!(m_read, 2); + assert_eq!(k_read, 50); + assert_eq!(sub_dim_read, 4); + + // Check expected pivots size: 16 header + 2*50*4*4 = 16 + 1600 = 1616 + assert_eq!(result.pivots_bytes.len(), 16 + 2 * 50 * 4 * 4); + + // Check compressed serialization. + assert!(result.compressed_bytes.len() > 8); + + // Verify compressed header: N=50, M=2 + let n_read = i32::from_ne_bytes(result.compressed_bytes[0..4].try_into().unwrap()); + let m_comp_read = i32::from_ne_bytes(result.compressed_bytes[4..8].try_into().unwrap()); + assert_eq!(n_read, 50); + assert_eq!(m_comp_read, 2); + + // Check expected compressed size: 8 header + 50*2 = 108 + assert_eq!(result.compressed_bytes.len(), 8 + 50 * 2); + } + + #[test] + fn test_pq_invalid_params() { + let vectors: Vec> = vec![vec![1.0, 2.0, 3.0, 4.0]]; + + // dim not divisible by num_subspaces + let err = train_and_encode(&vectors, 4, 3, 100, 5).unwrap_err(); + assert!(err.contains("Invalid PQ params")); + + // num_subspaces = 0 + let err = train_and_encode(&vectors, 4, 0, 100, 5).unwrap_err(); + assert!(err.contains("Invalid PQ params")); + + // empty vectors + let err = train_and_encode(&[], 4, 2, 100, 5).unwrap_err(); + assert!(err.contains("No vectors")); + } +} diff --git a/paimon-diskann/paimon-diskann-jni/src/test/java/org/apache/paimon/diskann/IndexTest.java b/paimon-diskann/paimon-diskann-jni/src/test/java/org/apache/paimon/diskann/IndexTest.java index 7b858e65725d..7fb44ae56dd3 100644 --- a/paimon-diskann/paimon-diskann-jni/src/test/java/org/apache/paimon/diskann/IndexTest.java +++ b/paimon-diskann/paimon-diskann-jni/src/test/java/org/apache/paimon/diskann/IndexTest.java @@ -25,7 +25,6 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.FloatBuffer; -import java.nio.LongBuffer; import java.util.Random; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -75,7 +74,7 @@ void testBasicOperations() { assertEquals(MetricType.L2, index.getMetricType()); // Add vectors with IDs - addVectorsWithIds(index, NUM_VECTORS, DIMENSION, 0); + addVectors(index, NUM_VECTORS, DIMENSION); assertEquals(NUM_VECTORS, index.getCount()); // Build the index @@ -103,21 +102,14 @@ void testBasicOperations() { } @Test - void testCustomIds() { + void testSequentialIds() { try (Index index = createIndex(MetricType.L2)) { - ByteBuffer vectorBuffer = createVectorBuffer(NUM_VECTORS, DIMENSION); - ByteBuffer idBuffer = Index.allocateIdBuffer(NUM_VECTORS); - LongBuffer longView = idBuffer.asLongBuffer(); - for (int i = 0; i < NUM_VECTORS; i++) { - longView.put(i, i * 100L); // Use custom IDs - } - - index.addWithIds(NUM_VECTORS, vectorBuffer, idBuffer); + addVectors(index, NUM_VECTORS, DIMENSION); assertEquals(NUM_VECTORS, index.getCount()); index.build(BUILD_LIST_SIZE); - // Search should return our custom IDs + // Search should return sequential IDs (0, 1, 2, ...) float[] queryVectors = createQueryVectors(1, DIMENSION); float[] distances = new float[K]; long[] labels = new long[K]; @@ -125,7 +117,9 @@ void testCustomIds() { index.search(1, queryVectors, K, SEARCH_LIST_SIZE, distances, labels); for (int i = 0; i < K; i++) { - assertTrue(labels[i] % 100 == 0, "Label should be a multiple of 100"); + assertTrue( + labels[i] >= 0 && labels[i] < NUM_VECTORS, + "Label " + labels[i] + " out of range"); } } } @@ -133,7 +127,7 @@ void testCustomIds() { @Test void testBatchSearch() { try (Index index = createIndex(MetricType.L2)) { - addVectorsWithIds(index, NUM_VECTORS, DIMENSION, 0); + addVectors(index, NUM_VECTORS, DIMENSION); index.build(BUILD_LIST_SIZE); int numQueries = 5; @@ -159,7 +153,7 @@ void testInnerProductMetric() { try (Index index = createIndex(MetricType.INNER_PRODUCT)) { assertEquals(MetricType.INNER_PRODUCT, index.getMetricType()); - addVectorsWithIds(index, NUM_VECTORS, DIMENSION, 0); + addVectors(index, NUM_VECTORS, DIMENSION); index.build(BUILD_LIST_SIZE); float[] queryVectors = createQueryVectors(1, DIMENSION); @@ -184,7 +178,7 @@ void testCosineMetric() { try (Index index = createIndex(MetricType.COSINE)) { assertEquals(MetricType.COSINE, index.getMetricType()); - addVectorsWithIds(index, NUM_VECTORS, DIMENSION, 0); + addVectors(index, NUM_VECTORS, DIMENSION); index.build(BUILD_LIST_SIZE); float[] queryVectors = createQueryVectors(1, DIMENSION); @@ -200,103 +194,31 @@ void testCosineMetric() { } } - @Test - void testSerialization() { - float[] queryVectors = createQueryVectors(1, DIMENSION); - long[] originalLabels = new long[K]; - float[] originalDistances = new float[K]; - - // Build a map of id -> vector for the in-memory reader - java.util.Map vectorMap = new java.util.HashMap<>(); - Random rng = new Random(42); - for (int i = 0; i < NUM_VECTORS; i++) { - float[] vec = new float[DIMENSION]; - for (int j = 0; j < DIMENSION; j++) { - vec[j] = rng.nextFloat(); - } - vectorMap.put((long) i, vec); - } - - // Create, populate, and search - try (Index index = createIndex(MetricType.L2)) { - addVectorsWithIds(index, NUM_VECTORS, DIMENSION, 0); - index.build(BUILD_LIST_SIZE); - - index.search(1, queryVectors, K, SEARCH_LIST_SIZE, originalDistances, originalLabels); - - // Serialize (graph + vectors format) - long serializeSize = index.serializeSize(); - assertTrue(serializeSize > 0); - - ByteBuffer serialized = - ByteBuffer.allocateDirect((int) serializeSize).order(ByteOrder.nativeOrder()); - long bytesWritten = index.serialize(serialized); - assertEquals(serializeSize, bytesWritten); - - // Convert to byte array - serialized.rewind(); - byte[] serializedBytes = new byte[(int) bytesWritten]; - serialized.get(serializedBytes); - - // Create a simple in-memory vector reader for testing. - // The Rust JNI layer invokes readVector(long) via reflection. - TestVectorReader reader = new TestVectorReader(vectorMap); - - // Deserialize via IndexSearcher and verify - try (IndexSearcher searcher = IndexSearcher.create(serializedBytes, reader)) { - assertEquals(DIMENSION, searcher.getDimension()); - - float[] deserializedDistances = new float[K]; - long[] deserializedLabels = new long[K]; - searcher.search( - 1, - queryVectors, - K, - SEARCH_LIST_SIZE, - deserializedDistances, - deserializedLabels); - - // Verify that search results are valid - for (int i = 0; i < K; i++) { - assertTrue( - deserializedLabels[i] >= 0 && deserializedLabels[i] < NUM_VECTORS, - "Label " + deserializedLabels[i] + " out of range"); - } - } - } - } - @Test void testSmallIndex() { int dim = 2; try (Index index = Index.create(dim, MetricType.L2, INDEX_TYPE_MEMORY, MAX_DEGREE, BUILD_LIST_SIZE)) { - // Add a few vectors + // Add a few vectors: [1,0], [0,1], [0.7,0.7] ByteBuffer vectorBuffer = Index.allocateVectorBuffer(3, dim); FloatBuffer floatView = vectorBuffer.asFloatBuffer(); floatView.put(0, 1.0f); - floatView.put(1, 0.0f); // [1, 0] + floatView.put(1, 0.0f); // position 0: [1, 0] floatView.put(2, 0.0f); - floatView.put(3, 1.0f); // [0, 1] + floatView.put(3, 1.0f); // position 1: [0, 1] floatView.put(4, 0.7f); - floatView.put(5, 0.7f); // [0.7, 0.7] - - ByteBuffer idBuffer = Index.allocateIdBuffer(3); - LongBuffer longView = idBuffer.asLongBuffer(); - longView.put(0, 10L); - longView.put(1, 20L); - longView.put(2, 30L); + floatView.put(5, 0.7f); // position 2: [0.7, 0.7] - index.addWithIds(3, vectorBuffer, idBuffer); + index.add(3, vectorBuffer); index.build(BUILD_LIST_SIZE); - // Query for [1, 0] - should find ID 10 as nearest + // Query for [1, 0] - should find position 0 as nearest float[] query = {1.0f, 0.0f}; float[] distances = new float[1]; long[] labels = new long[1]; index.search(1, query, 1, SEARCH_LIST_SIZE, distances, labels); - assertEquals(10L, labels[0], "Nearest to [1,0] should be ID 10"); + assertEquals(0L, labels[0], "Nearest to [1,0] should be position 0"); assertEquals(0.0f, distances[0], 1e-5f, "Distance to self should be ~0"); } } @@ -304,7 +226,7 @@ void testSmallIndex() { @Test void testSearchResultArrays() { try (Index index = createIndex(MetricType.L2)) { - addVectorsWithIds(index, 100, DIMENSION, 0); + addVectors(index, 100, DIMENSION); index.build(BUILD_LIST_SIZE); int numQueries = 3; @@ -333,12 +255,6 @@ void testBufferAllocationHelpers() { assertTrue(vectorBuffer.isDirect()); assertEquals(ByteOrder.nativeOrder(), vectorBuffer.order()); assertEquals(10 * DIMENSION * Float.BYTES, vectorBuffer.capacity()); - - // Test ID buffer allocation - ByteBuffer idBuffer = Index.allocateIdBuffer(10); - assertTrue(idBuffer.isDirect()); - assertEquals(ByteOrder.nativeOrder(), idBuffer.order()); - assertEquals(10 * Long.BYTES, idBuffer.capacity()); } @Test @@ -347,22 +263,20 @@ void testErrorHandling() { try (Index index = createIndex(MetricType.L2)) { ByteBuffer wrongSizeBuffer = ByteBuffer.allocateDirect(10).order(ByteOrder.nativeOrder()); - ByteBuffer idBuffer = Index.allocateIdBuffer(1); assertThrows( IllegalArgumentException.class, () -> { - index.addWithIds(1, wrongSizeBuffer, idBuffer); + index.add(1, wrongSizeBuffer); }); } // Test non-direct buffer try (Index index = createIndex(MetricType.L2)) { ByteBuffer heapBuffer = ByteBuffer.allocate(DIMENSION * Float.BYTES); - ByteBuffer idBuffer = Index.allocateIdBuffer(1); assertThrows( IllegalArgumentException.class, () -> { - index.addWithIds(1, heapBuffer, idBuffer); + index.add(1, heapBuffer); }); } @@ -389,7 +303,7 @@ void testErrorHandling() { @Test void testQueryVectorArrayValidation() { try (Index index = createIndex(MetricType.L2)) { - addVectorsWithIds(index, 10, DIMENSION, 0); + addVectors(index, 10, DIMENSION); index.build(BUILD_LIST_SIZE); // Query vectors array too small @@ -425,15 +339,10 @@ private Index createIndex(MetricType metricType) { return Index.create(DIMENSION, metricType, INDEX_TYPE_MEMORY, MAX_DEGREE, BUILD_LIST_SIZE); } - /** Add vectors with sequential IDs starting from {@code startId}. */ - private void addVectorsWithIds(Index index, int n, int d, long startId) { + /** Add random vectors to the index. */ + private void addVectors(Index index, int n, int d) { ByteBuffer vectorBuffer = createVectorBuffer(n, d); - ByteBuffer idBuffer = Index.allocateIdBuffer(n); - LongBuffer longView = idBuffer.asLongBuffer(); - for (int i = 0; i < n; i++) { - longView.put(i, startId + i); - } - index.addWithIds(n, vectorBuffer, idBuffer); + index.add(n, vectorBuffer); } /** Create a direct ByteBuffer with random vectors. */ @@ -458,23 +367,4 @@ private float[] createQueryVectors(int n, int d) { } return vectors; } - - /** - * Simple in-memory vector reader for testing. The Rust JNI layer invokes {@code - * readVector(long)} via reflection — no specific interface is required. - */ - static class TestVectorReader implements java.io.Closeable { - private final java.util.Map vectorMap; - - TestVectorReader(java.util.Map vectorMap) { - this.vectorMap = vectorMap; - } - - public float[] readVector(long vectorId) { - return vectorMap.get(vectorId); - } - - @Override - public void close() {} - } } From fdfa4b02a5d614ab8430a669c84016617ac5792c Mon Sep 17 00:00:00 2001 From: yantian Date: Sat, 14 Feb 2026 13:26:31 +0800 Subject: [PATCH 5/7] fix --- .../index/DiskAnnVectorGlobalIndexReader.java | 2 +- .../diskann/index/FileIOVectorReader.java | 179 +++++++--- .../apache/paimon/diskann/IndexSearcher.java | 24 +- .../src/main/native/src/lib.rs | 39 ++- .../main/native/src/paimon_fileio_provider.rs | 320 +++++++++++++----- 5 files changed, 424 insertions(+), 140 deletions(-) diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java index b9f40dd3317d..2b82d4591ac3 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java @@ -333,7 +333,7 @@ private void loadIndexAt(int position) throws IOException { GlobalIndexIOMeta dataIOMeta = new GlobalIndexIOMeta(dataPath, 0L, new byte[0]); SeekableInputStream vectorStream = fileReader.getInputStream(dataIOMeta); FileIOVectorReader vectorReader = - new FileIOVectorReader(vectorStream, meta.dim(), VECTOR_CACHE_SIZE); + new FileIOVectorReader(vectorStream, meta.dim(), meta.maxDegree()); // 3. Create DiskANN native searcher with on-demand graph + vector access. handle = diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOVectorReader.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOVectorReader.java index 079717d63a2b..110e47c20cef 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOVectorReader.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/FileIOVectorReader.java @@ -24,19 +24,22 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.util.Arrays; -import java.util.LinkedHashMap; -import java.util.Map; /** * Fetches vectors from a DiskANN data file through a Paimon {@link SeekableInputStream}. * *

    The underlying stream can be backed by any Paimon FileIO provider — local, HDFS, S3, OSS, etc. - * This class adds an LRU cache so that repeated reads for the same vector (common during DiskANN's - * beam search) do not trigger redundant I/O. * - *

    The Rust JNI layer invokes {@link #readVector(long)} via reflection during DiskANN's native - * beam search — no specific Java interface is required. + *

    The Rust JNI layer uses two access modes: + * + *

      + *
    • Single-vector zero-copy: {@link #loadVector(long)} reads a vector into a + * pre-allocated {@link ByteBuffer#allocateDirect DirectByteBuffer}. The Rust side reads + * floats directly from the native memory address — no {@code float[]} allocation and no JNI + * array copy. + *
    • Batch prefetch: {@link #readVectorsBatch(long[], int)} reads multiple vectors into a + * larger DirectByteBuffer in a single JNI call, reducing per-vector JNI round-trip overhead. + *
    * *

    Data file layout

    * @@ -55,59 +58,153 @@ public class FileIOVectorReader implements Closeable { /** Vector dimension. */ private final int dimension; - /** LRU cache: position → float[]. */ - private final LinkedHashMap cache; + /** Byte size of a single vector: {@code dimension * Float.BYTES}. */ + private final int vectorBytes; - /** Reusable byte buffer for reading a single vector. */ + /** Reusable heap byte buffer for stream I/O (stream API requires {@code byte[]}). */ private final byte[] readBuf; + /** + * Pre-allocated DirectByteBuffer for single-vector reads. Rust reads directly from its native + * address via {@code GetDirectBufferAddress} — zero JNI array copy. + */ + private final ByteBuffer directBuf; + + /** + * Pre-allocated DirectByteBuffer for batch reads. Holds up to {@code maxBatchSize} vectors + * packed sequentially. + */ + private final ByteBuffer batchBuf; + + /** Maximum number of vectors that fit in {@link #batchBuf}. */ + private final int maxBatchSize; + /** * Create a reader. * * @param input seekable input stream for the data file * @param dimension vector dimension - * @param cacheSize maximum number of cached vectors (0 disables caching) + * @param maxBatchSize maximum number of vectors in a batch read (typically max_degree) */ - public FileIOVectorReader(SeekableInputStream input, int dimension, int cacheSize) { + public FileIOVectorReader(SeekableInputStream input, int dimension, int maxBatchSize) { this.input = input; this.dimension = dimension; - this.readBuf = new byte[dimension * Float.BYTES]; - - final int cap = Math.max(cacheSize, 16); - this.cache = - new LinkedHashMap(cap, 0.75f, true) { - @Override - protected boolean removeEldestEntry(Map.Entry eldest) { - return size() > cap; - } - }; + this.vectorBytes = dimension * Float.BYTES; + this.readBuf = new byte[vectorBytes]; + this.maxBatchSize = Math.max(maxBatchSize, 1); + + // Single-vector DirectByteBuffer — Rust gets its native address once at init. + this.directBuf = ByteBuffer.allocateDirect(vectorBytes).order(ByteOrder.nativeOrder()); + + // Batch DirectByteBuffer — sized for maxBatchSize vectors. + this.batchBuf = + ByteBuffer.allocateDirect(this.maxBatchSize * vectorBytes) + .order(ByteOrder.nativeOrder()); } + // ------------------------------------------------------------------ + // DirectByteBuffer accessors (called by Rust JNI during init) + // ------------------------------------------------------------------ + + /** Return the single-vector DirectByteBuffer. Rust caches its native address. */ + public ByteBuffer getDirectBuffer() { + return directBuf; + } + + /** Return the batch DirectByteBuffer. Rust caches its native address. */ + public ByteBuffer getBatchBuffer() { + return batchBuf; + } + + /** Return the maximum batch size supported by {@link #batchBuf}. */ + public int getMaxBatchSize() { + return maxBatchSize; + } + + // ------------------------------------------------------------------ + // Single-vector zero-copy read (hot path during beam search) + // ------------------------------------------------------------------ + /** - * Read the vector associated with the given external ID. - * - *

    Called by the Rust JNI layer during DiskANN's native beam search. Returns a defensive - * copy — callers may freely modify the returned array without corrupting the cache. + * Read a single vector into the pre-allocated {@link #directBuf}. * - *

    The byte offset is computed as {@code position * dimension * Float.BYTES}. + *

    After this call returns {@code true}, the vector data is available in the DirectByteBuffer + * at offset 0. The Rust side reads floats directly from the native memory address. * - * @param position the 0-based position in the data file (int_id - 1 for user vectors) - * @return the float vector (a fresh copy), or {@code null} if position is negative + * @param position 0-based position in the data file (int_id − 1) + * @return {@code true} if the vector was read successfully, {@code false} if position is + * invalid */ - public float[] readVector(long position) { - // Start point (position = -1) is not in the data file. + public boolean loadVector(long position) { if (position < 0) { - return null; + return false; + } + long byteOffset = position * vectorBytes; + try { + input.seek(byteOffset); + readFully(input, readBuf); + } catch (IOException e) { + throw new RuntimeException( + "Failed to read vector at position " + position + " offset " + byteOffset, e); } + // Copy from heap byte[] into DirectByteBuffer (single memcpy, no float[] allocation). + directBuf.clear(); + directBuf.put(readBuf, 0, vectorBytes); + return true; + } + + // ------------------------------------------------------------------ + // Batch prefetch (reduces JNI call count) + // ------------------------------------------------------------------ - // 1. LRU cache hit — return a defensive copy. - float[] cached = cache.get(position); - if (cached != null) { - return Arrays.copyOf(cached, cached.length); + /** + * Read multiple vectors into the batch DirectByteBuffer in one JNI call. + * + *

    Vectors are packed sequentially in the batch buffer: vector i occupies bytes {@code [i * + * vectorBytes, (i+1) * vectorBytes)}. The Rust side reads all vectors from the native address + * after a single JNI round-trip. + * + * @param positions array of 0-based positions (int_id − 1 for each vector) + * @param count number of positions to read (must be ≤ {@link #maxBatchSize}) + * @return number of vectors successfully read (always equals {@code count} on success) + */ + public int readVectorsBatch(long[] positions, int count) { + int n = Math.min(count, maxBatchSize); + batchBuf.clear(); + for (int i = 0; i < n; i++) { + long byteOffset = positions[i] * vectorBytes; + try { + input.seek(byteOffset); + readFully(input, readBuf); + batchBuf.put(readBuf, 0, vectorBytes); + } catch (IOException e) { + throw new RuntimeException( + "Failed to batch-read vector at position " + + positions[i] + + " offset " + + byteOffset, + e); + } } + return n; + } + + // ------------------------------------------------------------------ + // Legacy read (kept for backward compatibility) + // ------------------------------------------------------------------ - // 2. Compute byte offset: sequential position. - long byteOffset = position * dimension * Float.BYTES; + /** + * Read a vector and return as {@code float[]}. This is the legacy path — prefer {@link + * #loadVector(long)} for the zero-copy hot path. + * + * @param position 0-based position in the data file + * @return the float vector, or {@code null} if position is negative + */ + public float[] readVector(long position) { + if (position < 0) { + return null; + } + long byteOffset = position * vectorBytes; try { input.seek(byteOffset); readFully(input, readBuf); @@ -115,20 +212,14 @@ public float[] readVector(long position) { throw new RuntimeException( "Failed to read vector at position " + position + " offset " + byteOffset, e); } - - // 3. Decode floats. float[] vector = new float[dimension]; ByteBuffer bb = ByteBuffer.wrap(readBuf).order(ByteOrder.nativeOrder()); bb.asFloatBuffer().get(vector); - - // 4. Store a separate copy in the cache so the returned array is independent. - cache.put(position, Arrays.copyOf(vector, vector.length)); return vector; } @Override public void close() throws IOException { - cache.clear(); input.close(); } diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java index 4ac2f45476c5..921994a2db06 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java @@ -43,18 +43,21 @@ public class IndexSearcher implements AutoCloseable { /** Native searcher handle (≥100 000, distinct from Index handles). */ private long nativeHandle; - /** Resources to close when this searcher is closed. */ - private final Closeable[] closeables; + private final Closeable graphReader; + + private final Closeable vectorReader; /** Vector dimension. */ private final int dimension; private volatile boolean closed = false; - private IndexSearcher(long nativeHandle, int dimension, Closeable... closeables) { + private IndexSearcher( + long nativeHandle, int dimension, Closeable graphReader, Closeable vectorReader) { this.nativeHandle = nativeHandle; this.dimension = dimension; - this.closeables = closeables; + this.graphReader = graphReader; + this.vectorReader = vectorReader; } /** @@ -135,12 +138,15 @@ public synchronized void close() { DiskAnnNative.indexDestroySearcher(nativeHandle); nativeHandle = 0; } - for (Closeable c : closeables) { - try { - c.close(); - } catch (Exception e) { - // best-effort + try { + if (graphReader != null) { + graphReader.close(); + } + if (vectorReader != null) { + vectorReader.close(); } + } catch (Exception e) { + // best-effort } } } diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs index 0a25552ed0d7..4b179b201223 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs @@ -787,10 +787,44 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("get JVM: {}", e)); return 0; } }; + // ---- Obtain DirectByteBuffer native pointers from the vector reader ---- + + // Single-vector DirectByteBuffer: getDirectBuffer() → ByteBuffer + let single_buf_ptr: *mut f32 = { + let buf_obj = match env.call_method(&vector_reader, "getDirectBuffer", "()Ljava/nio/ByteBuffer;", &[]) { + Ok(v) => match v.l() { Ok(o) => o, Err(_) => { let _ = env.throw_new("java/lang/RuntimeException", "Bad return from getDirectBuffer"); return 0; } }, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("getDirectBuffer: {}", e)); return 0; } + }; + let byte_buf = jni::objects::JByteBuffer::from(buf_obj); + match env.get_direct_buffer_address(&byte_buf) { + Ok(ptr) => ptr as *mut f32, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("GetDirectBufferAddress (single): {}", e)); return 0; } + } + }; + + // Batch DirectByteBuffer: getBatchBuffer() → ByteBuffer + let batch_buf_ptr: *mut f32 = { + let buf_obj = match env.call_method(&vector_reader, "getBatchBuffer", "()Ljava/nio/ByteBuffer;", &[]) { + Ok(v) => match v.l() { Ok(o) => o, Err(_) => { let _ = env.throw_new("java/lang/RuntimeException", "Bad return from getBatchBuffer"); return 0; } }, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("getBatchBuffer: {}", e)); return 0; } + }; + let byte_buf = jni::objects::JByteBuffer::from(buf_obj); + match env.get_direct_buffer_address(&byte_buf) { + Ok(ptr) => ptr as *mut f32, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("GetDirectBufferAddress (batch): {}", e)); return 0; } + } + }; + + // Max batch size from the vector reader. + let max_batch_size: usize = match env.call_method(&vector_reader, "getMaxBatchSize", "()I", &[]) { + Ok(v) => match v.i() { Ok(i) => i as usize, Err(_) => max_degree }, + Err(_) => max_degree, + }; + // Start point is not stored in data file; use a dummy vector. let start_vec = vec![1.0f32; dim]; - // Build the FileIOProvider with on-demand graph reading. + // Build the FileIOProvider with on-demand graph reading and zero-copy vector access. let provider = FileIOProvider::new_with_readers( count, start_id, @@ -801,6 +835,9 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS dim, metric_type, max_degree, + single_buf_ptr, + batch_buf_ptr, + max_batch_size, ); // Build DiskANNIndex config. diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs index 6db3e727f6f9..00b9dcd81b0b 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs @@ -25,11 +25,19 @@ //! //! - **Graph**: read through `FileIOGraphReader.readNeighbors(int)`, which //! reads from a `SeekableInputStream` over the `.index` file. -//! - **Vectors**: read through `FileIOVectorReader.readVector(long)`, which -//! reads from a `SeekableInputStream` over the `.data` file. +//! - **Vectors**: read through `FileIOVectorReader.loadVector(long)` (zero-copy +//! via DirectByteBuffer) or `readVectorsBatch(long[], int)` (batch prefetch). //! -//! Frequently accessed neighbors and vectors are cached in a `DashMap` and -//! an LRU cache respectively, to reduce FileIO/JNI round-trips. +//! Performance optimizations: +//! +//! - **Zero-copy vector reads**: `loadVector` writes into a pre-allocated +//! DirectByteBuffer. The Rust side reads floats directly from the native +//! memory address — no `float[]` allocation, no JNI array copy. +//! - **Batch prefetch**: When a node's neighbors are fetched, all neighbor +//! vectors are batch-prefetched in a single JNI call, populating the +//! provider-level `vector_cache`. Subsequent `get_element` calls hit cache. +//! - **Graph cache**: A `DashMap` lazily caches graph entries to reduce +//! repeated FileIO reads. use std::collections::HashMap; @@ -41,7 +49,6 @@ use diskann::{ANNError, ANNResult}; use diskann_vector::distance::Metric; use jni::objects::GlobalRef; -use jni::sys::jlong; use jni::JavaVM; use crate::map_metric; @@ -74,43 +81,6 @@ impl From for ANNError { diskann::always_escalate!(FileIOProviderError); -// ======================== LRU Cache ======================== - -/// Tiny LRU cache for recently fetched vectors to reduce FileIO/JNI round-trips. -struct VectorCache { - map: HashMap>, - order: Vec, - capacity: usize, -} - -impl VectorCache { - fn new(capacity: usize) -> Self { - Self { - map: HashMap::with_capacity(capacity), - order: Vec::with_capacity(capacity), - capacity, - } - } - - fn get(&self, id: u32) -> Option<&[f32]> { - self.map.get(&id).map(|v| &**v) - } - - fn put(&mut self, id: u32, vec: Box<[f32]>) { - if self.map.contains_key(&id) { - return; - } - if self.order.len() >= self.capacity { - if let Some(evicted) = self.order.first().copied() { - self.order.remove(0); - self.map.remove(&evicted); - } - } - self.order.push(id); - self.map.insert(id, vec); - } -} - // ======================== Graph Term ======================== /// One entry in the graph cache: its neighbor list. @@ -124,11 +94,17 @@ pub struct GraphTerm { /// /// Graph neighbors and vectors are read on demand from FileIO-backed storage /// (local, HDFS, S3, OSS, etc.) via JNI callbacks to Java reader objects. -/// A `DashMap` lazily caches graph entries to reduce repeated FileIO reads. +/// +/// Two levels of caching reduce FileIO/JNI round-trips: +/// - **Graph**: `DashMap` (lazy, write-once, unbounded). +/// - **Vectors**: `DashMap>` populated by batch prefetch +/// after each neighbor expansion, plus per-search LRU in `FileIOAccessor`. pub struct FileIOProvider { /// Graph cache: internal_id → { neighbors }. - /// Acts as a lazy cache — entries are populated on first access from FileIO. graph: DashMap, + /// Provider-level vector cache populated by batch prefetch. + /// Keyed by internal node ID (u32). + vector_cache: DashMap>, /// Total number of nodes (start point + user vectors). num_nodes: usize, /// Start-point IDs and their vectors (always kept in memory). @@ -138,7 +114,6 @@ pub struct FileIOProvider { /// Global reference to the Java vector reader object (`FileIOVectorReader`). reader_ref: GlobalRef, /// Global reference to the Java graph reader object (`FileIOGraphReader`). - /// When set, graph neighbors are fetched on demand via JNI callbacks. graph_reader_ref: Option, /// Vector dimension. dim: usize, @@ -146,6 +121,14 @@ pub struct FileIOProvider { metric: Metric, /// Max degree. max_degree: usize, + /// Native memory address of the single-vector DirectByteBuffer. + /// Points to `dim` floats. Valid for the lifetime of the Java reader. + single_buf_ptr: *mut f32, + /// Native memory address of the batch DirectByteBuffer. + /// Points to `max_batch_size * dim` floats. + batch_buf_ptr: *mut f32, + /// Maximum number of vectors in one batch read. + max_batch_size: usize, } impl std::fmt::Debug for FileIOProvider { @@ -155,19 +138,24 @@ impl std::fmt::Debug for FileIOProvider { .field("metric", &self.metric) .field("max_degree", &self.max_degree) .field("graph_size", &self.graph.len()) + .field("vector_cache_size", &self.vector_cache.len()) .finish() } } // SAFETY: JavaVM is Send+Sync, GlobalRef is Send+Sync. +// Raw pointers are stable (backed by Java DirectByteBuffer kept alive by GlobalRef). +// All access is serialized by single-threaded tokio runtime. unsafe impl Send for FileIOProvider {} unsafe impl Sync for FileIOProvider {} impl FileIOProvider { - /// Build a search-only provider with on-demand graph reading (no pre-loaded graph data). + /// Build a search-only provider with on-demand graph reading and zero-copy + /// vector access via DirectByteBuffers. /// - /// The graph `DashMap` starts empty and acts as a lazy cache — entries are populated - /// on first access via FileIO through `graphReader.readNeighbors(int)`. + /// `single_buf_ptr` and `batch_buf_ptr` are native addresses obtained via + /// JNI `GetDirectBufferAddress` on the Java reader's DirectByteBuffers. + #[allow(clippy::too_many_arguments)] pub fn new_with_readers( num_nodes: usize, start_id: u32, @@ -178,14 +166,19 @@ impl FileIOProvider { dim: usize, metric_type: i32, max_degree: usize, + single_buf_ptr: *mut f32, + batch_buf_ptr: *mut f32, + max_batch_size: usize, ) -> Self { let graph = DashMap::new(); + let vector_cache = DashMap::new(); let mut start_points = HashMap::new(); start_points.insert(start_id, start_vec); Self { graph, + vector_cache, num_nodes, start_points, jvm, @@ -194,6 +187,9 @@ impl FileIOProvider { dim, metric: map_metric(metric_type), max_degree, + single_buf_ptr, + batch_buf_ptr, + max_batch_size, } } @@ -205,8 +201,10 @@ impl FileIOProvider { self.metric } + // ---- Graph I/O ---- + /// Fetch neighbor list from FileIO-backed storage via JNI callback to - /// `graphReader.readNeighbors(int)`. Returns None if graphReader is not set. + /// `graphReader.readNeighbors(int)`. fn fetch_neighbors(&self, int_id: u32) -> Result>, FileIOProviderError> { let graph_ref = match &self.graph_reader_ref { Some(r) => r, @@ -258,9 +256,13 @@ impl FileIOProvider { Ok(Some(buf.into_iter().map(|v| v as u32).collect())) } - /// Fetch a vector from FileIO-backed storage via JNI callback to - /// `vectorReader.readVector(long)`. The `position` is the 0-based index - /// in the data file (position = int_id - 1). + // ---- Vector I/O (zero-copy via DirectByteBuffer) ---- + + /// Fetch a single vector via `loadVector(long)` and read from DirectByteBuffer. + /// + /// The Java method writes the vector into the pre-allocated DirectByteBuffer. + /// We then read floats directly from the native address — no `float[]` + /// allocation and no JNI array copy. fn fetch_vector(&self, position: i64) -> Result>, FileIOProviderError> { let mut env = self .jvm @@ -269,45 +271,114 @@ impl FileIOProvider { let result = env.call_method( &self.reader_ref, - "readVector", - "(J)[F", - &[jni::objects::JValue::Long(position as jlong)], + "loadVector", + "(J)Z", + &[jni::objects::JValue::Long(position)], ); - let ret_val = match result { - Ok(v) => v, + let success = match result { + Ok(v) => match v.z() { + Ok(b) => b, + Err(_) => false, + }, Err(e) => { let _ = env.exception_clear(); return Err(FileIOProviderError::JniCallFailed(format!( - "readVector({}) failed: {}", + "loadVector({}) failed: {}", position, e ))); } }; - let obj = match ret_val.l() { - Ok(o) => o, - Err(_) => return Ok(None), + if !success { + return Ok(None); + } + + // Read floats directly from the DirectByteBuffer native address. + // SAFETY: single_buf_ptr is valid (backed by Java DirectByteBuffer kept alive + // by GlobalRef), and access is serialized (single-threaded tokio runtime). + let vec = unsafe { + let slice = std::slice::from_raw_parts(self.single_buf_ptr, self.dim); + slice.to_vec() }; - if obj.is_null() { - return Ok(None); + Ok(Some(vec)) + } + + /// Batch-prefetch vectors into the provider-level `vector_cache`. + /// + /// Calls `readVectorsBatch(long[], int)` once via JNI, then reads all vectors + /// from the batch DirectByteBuffer native address. Each vector is inserted + /// into `vector_cache` keyed by its internal node ID. + /// + /// `ids` contains internal node IDs (not positions). Position = id − 1. + fn prefetch_vectors(&self, ids: &[u32]) -> Result<(), FileIOProviderError> { + if ids.is_empty() || self.batch_buf_ptr.is_null() { + return Ok(()); } - // Convert JFloatArray → Vec. - let float_array = jni::objects::JFloatArray::from(obj); - let len = env - .get_array_length(&float_array) - .map_err(|e| FileIOProviderError::JniCallFailed(format!("get_array_length: {}", e)))? - as usize; + let count = std::cmp::min(ids.len(), self.max_batch_size); + + let mut env = self + .jvm + .attach_current_thread() + .map_err(|e| FileIOProviderError::JniCallFailed(format!("attach failed: {}", e)))?; - let mut buf = vec![0f32; len]; - env.get_float_array_region(&float_array, 0, &mut buf) - .map_err(|e| { - FileIOProviderError::JniCallFailed(format!("get_float_array_region: {}", e)) - })?; + // Build Java long[] of positions (position = int_id − 1). + let positions: Vec = ids[..count].iter().map(|&id| (id as i64) - 1).collect(); + let java_positions = env + .new_long_array(count as i32) + .map_err(|e| FileIOProviderError::JniCallFailed(format!("new_long_array: {}", e)))?; + env.set_long_array_region(&java_positions, 0, &positions) + .map_err(|e| FileIOProviderError::JniCallFailed(format!("set_long_array_region: {}", e)))?; + + // Single JNI call: readVectorsBatch(long[], int) → int + // SAFETY: JLongArray wraps a JObject; we reinterpret the raw pointer. + let positions_obj = unsafe { + jni::objects::JObject::from_raw(java_positions.as_raw()) + }; + let result = env.call_method( + &self.reader_ref, + "readVectorsBatch", + "([JI)I", + &[ + jni::objects::JValue::Object(&positions_obj), + jni::objects::JValue::Int(count as i32), + ], + ); + // Prevent double-free: positions_obj shares the raw handle with java_positions. + std::mem::forget(positions_obj); + + let read_count = match result { + Ok(v) => match v.i() { + Ok(n) => n as usize, + Err(_) => 0, + }, + Err(e) => { + let _ = env.exception_clear(); + return Err(FileIOProviderError::JniCallFailed(format!( + "readVectorsBatch failed: {}", + e + ))); + } + }; - Ok(Some(buf)) + // Read vectors from batch DirectByteBuffer native address and populate cache. + // SAFETY: batch_buf_ptr is valid, access is serialized. + for i in 0..read_count { + let int_id = ids[i]; + if self.vector_cache.contains_key(&int_id) { + continue; // already cached + } + let offset = i * self.dim; + let vec = unsafe { + let slice = std::slice::from_raw_parts(self.batch_buf_ptr.add(offset), self.dim); + slice.to_vec() + }; + self.vector_cache.insert(int_id, vec.into_boxed_slice()); + } + + Ok(()) } } @@ -393,9 +464,11 @@ impl provider::NeighborAccessor for FileIONeighborAccessor<'_> { id: Self::Id, neighbors: &mut AdjacencyList, ) -> ANNResult { - // 1. Try cached graph (populated upfront or cached from previous FileIO reads). + // 1. Try cached graph. if let Some(term) = self.provider.graph.get(&id) { neighbors.overwrite_trusted(&term.neighbors); + // Batch-prefetch neighbor vectors that aren't cached yet. + self.prefetch_neighbor_vectors(&term.neighbors); return Ok(self); } @@ -406,7 +479,9 @@ impl provider::NeighborAccessor for FileIONeighborAccessor<'_> { let adj = AdjacencyList::from_iter_untrusted(neighbor_ids.iter().copied()); neighbors.overwrite_trusted(&adj); // Cache in the DashMap for subsequent accesses. - self.provider.graph.insert(id, GraphTerm { neighbors: adj }); + self.provider.graph.insert(id, GraphTerm { neighbors: adj.clone() }); + // Batch-prefetch neighbor vectors. + self.prefetch_neighbor_vectors(&adj); return Ok(self); } } @@ -415,6 +490,32 @@ impl provider::NeighborAccessor for FileIONeighborAccessor<'_> { } } +impl FileIONeighborAccessor<'_> { + /// Batch-prefetch vectors for neighbors that aren't already in the vector cache. + /// + /// This is the key optimization: after discovering neighbors via graph I/O, + /// we batch-fetch all their vectors in a single JNI call. Subsequent + /// `get_element` calls for these neighbors will hit the provider-level cache. + fn prefetch_neighbor_vectors(&self, adj: &AdjacencyList) { + // Collect uncached neighbor IDs (skip start points and already-cached). + let uncached: Vec = adj + .iter() + .copied() + .filter(|&nid| { + !self.provider.start_points.contains_key(&nid) + && !self.provider.vector_cache.contains_key(&nid) + }) + .collect(); + + if uncached.is_empty() { + return; + } + + // Best-effort: don't propagate errors from prefetch. + let _ = self.provider.prefetch_vectors(&uncached); + } +} + impl provider::NeighborAccessorMut for FileIONeighborAccessor<'_> { async fn set_neighbors(self, id: Self::Id, neighbors: &[Self::Id]) -> ANNResult { match self.provider.graph.get_mut(&id) { @@ -450,9 +551,12 @@ impl provider::DefaultAccessor for FileIOProvider { // ======================== Accessor ======================== -/// Accessor that fetches vectors from FileIO-backed storage via JNI callback. +/// Accessor that fetches vectors with three cache levels: /// -/// Keeps a local buffer and an LRU cache to reduce FileIO/JNI round-trips. +/// 1. **Start-point** (always in memory) +/// 2. **Provider-level `vector_cache`** (populated by batch prefetch) +/// 3. **Per-search LRU cache** (local to this accessor) +/// 4. **DirectByteBuffer I/O** (fallback: single JNI call + zero-copy read) pub struct FileIOAccessor<'a> { provider: &'a FileIOProvider, buffer: Box<[f32]>, @@ -490,26 +594,34 @@ impl provider::Accessor for FileIOAccessor<'_> { return Ok(&*self.buffer); } - // 2. Check LRU cache. + // 2. Provider-level vector cache (populated by batch prefetch). + if let Some(cached) = self.provider.vector_cache.get(&id) { + self.buffer.copy_from_slice(&cached); + return Ok(&*self.buffer); + } + + // 3. Per-search LRU cache. if let Some(cached) = self.cache.get(id) { self.buffer.copy_from_slice(cached); return Ok(&*self.buffer); } - // 3. Fetch from FileIO-backed storage via FileIOVectorReader.readVector(position). - // position = int_id - 1 (start point is int_id=0, user vectors start at 1). + // 4. Fetch via DirectByteBuffer I/O (single JNI call + zero-copy read). let position = (id as i64) - 1; - let fetched = self.provider.fetch_vector(position)?; match fetched { Some(vec) if vec.len() == self.provider.dim() => { self.buffer.copy_from_slice(&vec); + // Populate both caches. + self.provider + .vector_cache + .insert(id, vec.clone().into_boxed_slice()); self.cache.put(id, vec.into_boxed_slice()); Ok(&*self.buffer) } Some(vec) => Err(FileIOProviderError::JniCallFailed(format!( - "readVector({}) returned {} floats, expected {}", + "loadVector({}) returned {} floats, expected {}", position, vec.len(), self.provider.dim() @@ -602,7 +714,7 @@ impl glue::SearchStrategy for FileIOStrategy { provider: &'a FileIOProvider, _context: &'a FileIOContext, ) -> Result, diskann::error::Infallible> { - // Cache up to 1024 recently fetched vectors to reduce FileIO round-trips. + // Per-search LRU as last-resort cache (most hits come from vector_cache). Ok(FileIOAccessor::new(provider, 1024)) } @@ -655,3 +767,41 @@ impl<'a> glue::AsElement<&'a [f32]> for FileIOAccessor<'a> { std::future::ready(Ok(vector)) } } + +// ======================== VectorCache (per-search LRU) ======================== + +/// Tiny LRU cache for per-search vector access. Most hits should come from the +/// provider-level `vector_cache`; this is a last-resort fallback. +struct VectorCache { + map: HashMap>, + order: Vec, + capacity: usize, +} + +impl VectorCache { + fn new(capacity: usize) -> Self { + Self { + map: HashMap::with_capacity(capacity), + order: Vec::with_capacity(capacity), + capacity, + } + } + + fn get(&self, id: u32) -> Option<&[f32]> { + self.map.get(&id).map(|v| &**v) + } + + fn put(&mut self, id: u32, vec: Box<[f32]>) { + if self.map.contains_key(&id) { + return; + } + if self.order.len() >= self.capacity { + if let Some(evicted) = self.order.first().copied() { + self.order.remove(0); + self.map.remove(&evicted); + } + } + self.order.push(id); + self.map.insert(id, vec); + } +} From d95a494b1597c9077f75f9a8bc6475f92b131bb1 Mon Sep 17 00:00:00 2001 From: yantian Date: Sat, 14 Feb 2026 15:08:54 +0800 Subject: [PATCH 6/7] fix search did not use pq --- .../index/DiskAnnVectorGlobalIndexReader.java | 44 +++- .../apache/paimon/diskann/DiskAnnNative.java | 16 +- .../apache/paimon/diskann/IndexSearcher.java | 19 +- .../src/main/native/src/lib.rs | 195 ++++++++++++++++-- .../main/native/src/paimon_fileio_provider.rs | 170 +++++++++++++-- 5 files changed, 404 insertions(+), 40 deletions(-) diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java index 2b82d4591ac3..184e442e0240 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java @@ -335,11 +335,22 @@ private void loadIndexAt(int position) throws IOException { FileIOVectorReader vectorReader = new FileIOVectorReader(vectorStream, meta.dim(), meta.maxDegree()); - // 3. Create DiskANN native searcher with on-demand graph + vector access. + // 3. Load PQ files into memory for in-memory approximate distance computation. + // Beam search uses PQ-reconstructed vectors; only top-K candidates are + // re-ranked with full-precision vectors from disk. + byte[] pqPivots = loadCompanionFile(ioMeta, meta.pqPivotsFileName()); + byte[] pqCompressed = loadCompanionFile(ioMeta, meta.pqCompressedFileName()); + + // 4. Create DiskANN native searcher with on-demand graph + vector access + PQ. handle = new DiskAnnSearchHandle( IndexSearcher.createFromReaders( - graphReader, vectorReader, meta.dim(), meta.minId())); + graphReader, + vectorReader, + meta.dim(), + meta.minId(), + pqPivots, + pqCompressed)); if (handles.size() <= position) { while (handles.size() < position) { @@ -388,6 +399,35 @@ public void close() throws IOException { } } + /** + * Load a companion file (e.g. PQ pivots/compressed) relative to the index file. + * + * @return the file contents as byte[], or null if the file name is empty or the file does not + * exist. + */ + private byte[] loadCompanionFile(GlobalIndexIOMeta indexIOMeta, String fileName) + throws IOException { + if (fileName == null || fileName.isEmpty()) { + return null; + } + Path filePath = new Path(indexIOMeta.filePath().getParent(), fileName); + GlobalIndexIOMeta fileMeta = new GlobalIndexIOMeta(filePath, 0L, new byte[0]); + try (SeekableInputStream in = fileReader.getInputStream(fileMeta)) { + // Read the entire file into a byte array. + java.io.ByteArrayOutputStream baos = new java.io.ByteArrayOutputStream(4096); + byte[] buf = new byte[8192]; + int n; + while ((n = in.read(buf)) >= 0) { + baos.write(buf, 0, n); + } + byte[] data = baos.toByteArray(); + return data.length > 0 ? data : null; + } catch (Exception e) { + // PQ files are optional — if missing, fall back to full-precision search. + return null; + } + } + private static class ScoredRow { final long rowId; final float score; diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java index 861dec172e87..035f6e6de5b5 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/DiskAnnNative.java @@ -101,16 +101,26 @@ static native void indexSearch( *

  • Graph: the Rust side calls {@code graphReader.readNeighbors(int)} via JNI to * fetch neighbor lists on demand during beam search. It also calls getter methods ({@code * getDimension()}, {@code getCount()}, {@code getStartId()}) during initialization. - *
  • Vectors: the Rust side calls {@code vectorReader.readVector(long)} via JNI. + *
  • Vectors: the Rust side calls {@code vectorReader.loadVector(long)} via JNI + * (zero-copy via DirectByteBuffer). + *
  • PQ: the PQ codebook and compressed codes are loaded into native memory for + * in-memory approximate distance computation during beam search. Only the final top-K + * candidates are re-ranked with full-precision vectors from disk. * * * @param graphReader a Java object providing graph structure on demand. - * @param vectorReader a Java object with a {@code float[] readVector(long)} method. + * @param vectorReader a Java object with a {@code loadVector(long)} method. * @param minExtId minimum external ID for this index (for int_id → ext_id conversion). + * @param pqPivots serialized PQ codebook (pivots), or null/empty to disable PQ. + * @param pqCompressed serialized PQ compressed codes, or null/empty to disable PQ. * @return a searcher handle (>= 100 000) for use with {@link #indexSearchWithReader}. */ static native long indexCreateSearcherFromReaders( - Object graphReader, Object vectorReader, long minExtId); + Object graphReader, + Object vectorReader, + long minExtId, + byte[] pqPivots, + byte[] pqCompressed); /** * Search on a searcher handle created by {@link #indexCreateSearcherFromReaders}. diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java index 921994a2db06..1c15436e59c8 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/IndexSearcher.java @@ -68,21 +68,34 @@ private IndexSearcher( * *
      *
    • {@code graphReader.readNeighbors(int)} for neighbor lists during beam search - *
    • {@code vectorReader.readVector(long)} for full-precision vectors during beam search + *
    • {@code vectorReader.loadVector(long)} for full-precision vectors (zero-copy via + * DirectByteBuffer) *
    * + *

    When PQ data is provided, beam search uses PQ-reconstructed vectors for approximate + * distance computation (fully in-memory), and only the final top-K candidates are re-ranked + * with full-precision vectors from disk I/O. + * *

    Both readers must implement {@link Closeable}. * * @param graphReader a graph reader object (e.g. {@code FileIOGraphReader}). * @param vectorReader a vector reader object (e.g. {@code FileIOVectorReader}). * @param dimension the vector dimension (from {@code DiskAnnIndexMeta}). * @param minExtId minimum external ID for this index (for int_id → ext_id conversion). + * @param pqPivots serialized PQ codebook, or null to disable PQ-accelerated search. + * @param pqCompressed serialized PQ compressed codes, or null to disable. * @return a new IndexSearcher */ public static IndexSearcher createFromReaders( - Closeable graphReader, Closeable vectorReader, int dimension, long minExtId) { + Closeable graphReader, + Closeable vectorReader, + int dimension, + long minExtId, + byte[] pqPivots, + byte[] pqCompressed) { long handle = - DiskAnnNative.indexCreateSearcherFromReaders(graphReader, vectorReader, minExtId); + DiskAnnNative.indexCreateSearcherFromReaders( + graphReader, vectorReader, minExtId, pqPivots, pqCompressed); return new IndexSearcher(handle, dimension, graphReader, vectorReader); } diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs index 4b179b201223..332cb9ff519e 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs @@ -584,6 +584,55 @@ struct SearcherState { dimension: i32, /// Minimum external ID for this index. ext_id = min_ext_id + (int_id - 1). min_ext_id: i64, + /// Reranking context: used after PQ-approximate beam search to compute + /// exact distances for the top-K candidates with full-precision vectors. + rerank: Option, +} + +/// Context for post-search reranking with full-precision vectors from disk. +/// +/// When PQ is enabled, beam search uses PQ-approximate distances (in-memory). +/// After search, we re-read full vectors for the top-K candidates and recompute +/// exact distances to produce the final ranking. +struct RerankContext { + /// Separate JVM handle for reranking (cheap clone of the pointer). + jvm: jni::JavaVM, + /// Separate GlobalRef to the vector reader (same Java object, different ref). + reader_ref: jni::objects::GlobalRef, + /// Native address of the single-vector DirectByteBuffer. + single_buf_ptr: *mut f32, + /// Vector dimension. + dim: usize, + /// Distance metric type (0=L2, 1=IP, 2=Cosine). + metric_type: i32, +} + +// SAFETY: same justification as FileIOProvider — JavaVM and GlobalRef are +// Send+Sync, raw pointer access is serialized by single-threaded runtime. +unsafe impl Send for RerankContext {} +unsafe impl Sync for RerankContext {} + +/// Compute exact distance between two vectors. +fn compute_exact_distance(a: &[f32], b: &[f32], metric_type: i32) -> f32 { + match metric_type { + METRIC_INNER_PRODUCT => { + // Negative inner product (larger IP = more similar → smaller distance). + let dot: f32 = a.iter().zip(b).map(|(x, y)| x * y).sum(); + -dot + } + METRIC_COSINE => { + // 1 − cos_sim + let dot: f32 = a.iter().zip(b).map(|(x, y)| x * y).sum(); + let norm_a: f32 = a.iter().map(|x| x * x).sum::().sqrt(); + let norm_b: f32 = b.iter().map(|x| x * x).sum::().sqrt(); + let denom = norm_a * norm_b; + if denom < 1e-30 { 1.0 } else { 1.0 - dot / denom } + } + _ => { + // Squared L2 distance. + a.iter().zip(b).map(|(x, y)| { let d = x - y; d * d }).sum() + } + } } struct SearcherRegistry { @@ -737,13 +786,14 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSeriali // ======================== indexCreateSearcherFromReaders ======================== /// Create a search-only handle from two on-demand Java readers: one for graph -/// structure and one for vectors. +/// structure and one for vectors, with optional PQ data for in-memory +/// approximate distance computation during beam search. /// -/// `graphReader`: Java object with `readNeighbors(int)`, `getDimension()`, -/// `getCount()`, `getStartId()`, `getMaxDegree()`, -/// `getBuildListSize()`, `getMetricValue()`. -/// `vectorReader`: Java object with `readVector(long)`. -/// `min_ext_id`: Minimum external ID for int_id → ext_id conversion. +/// `graphReader`: Java object with `readNeighbors(int)`, `getDimension()`, etc. +/// `vectorReader`: Java object with `loadVector(long)`, DirectByteBuffer accessors. +/// `min_ext_id`: Minimum external ID for int_id → ext_id conversion. +/// `pq_pivots`: Serialized PQ codebook (byte[]), or null to disable PQ. +/// `pq_compressed`: Serialized PQ compressed codes (byte[]), or null. /// /// Returns a searcher handle (≥100000) for use with `indexSearchWithReader`. #[no_mangle] @@ -753,6 +803,8 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS graph_reader: JObject<'local>, vector_reader: JObject<'local>, min_ext_id: jlong, + pq_pivots: JObject<'local>, + pq_compressed: JObject<'local>, ) -> jlong { // Helper to call int-returning methods on graphReader. macro_rules! call_int { @@ -821,6 +873,62 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS Err(_) => max_degree, }; + // ---- Deserialize PQ data (if provided) ---- + + let pq_state = if !pq_pivots.is_null() && !pq_compressed.is_null() { + let pivots_bytes: Vec = match env.convert_byte_array( + jni::objects::JByteArray::from(pq_pivots), + ) { + Ok(b) if !b.is_empty() => b, + _ => Vec::new(), + }; + let compressed_bytes: Vec = match env.convert_byte_array( + jni::objects::JByteArray::from(pq_compressed), + ) { + Ok(b) if !b.is_empty() => b, + _ => Vec::new(), + }; + + if !pivots_bytes.is_empty() && !compressed_bytes.is_empty() { + match paimon_fileio_provider::PQState::deserialize(&pivots_bytes, &compressed_bytes) { + Ok(pq) => Some(pq), + Err(e) => { + // PQ deserialization failed — fall back to full-precision search. + eprintln!("PQ deserialization failed (falling back to full search): {}", e); + None + } + } + } else { + None + } + } else { + None + }; + + let pq_enabled = pq_state.is_some(); + + // ---- Create reranking context (separate GlobalRef + JVM for post-search reranking) ---- + + let rerank = if pq_enabled { + let rerank_reader_ref = match env.new_global_ref(&vector_reader) { + Ok(g) => g, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("rerank reader ref: {}", e)); return 0; } + }; + let rerank_jvm = match env.get_java_vm() { + Ok(vm) => vm, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("rerank JVM: {}", e)); return 0; } + }; + Some(RerankContext { + jvm: rerank_jvm, + reader_ref: rerank_reader_ref, + single_buf_ptr, + dim, + metric_type, + }) + } else { + None + }; + // Start point is not stored in data file; use a dummy vector. let start_vec = vec![1.0f32; dim]; @@ -838,6 +946,7 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS single_buf_ptr, batch_buf_ptr, max_batch_size, + pq_state, ); // Build DiskANNIndex config. @@ -871,6 +980,7 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS runtime, dimension, min_ext_id, + rerank, }; match searcher_registry().lock() { @@ -967,15 +1077,70 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearchW }; let rc = stats.result_count as usize; - let mut cnt = 0; - for ri in 0..rc { - if cnt >= top_k { break; } - let nb = &neighbors[ri]; - if nb.id == START_POINT_ID { continue; } - let idx = qi * top_k + cnt; - result_lbl[idx] = state.min_ext_id + (nb.id as i64) - 1; - result_dist[idx] = nb.distance; - cnt += 1; + + // ---- Reranking: replace PQ-approximate distances with exact distances ---- + // When PQ is enabled, beam search used PQ-reconstructed vectors for distance + // computation. Now we read full-precision vectors from disk for the top + // candidates and recompute exact distances, then re-sort. + if let Some(ref rerank_ctx) = state.rerank { + // Collect valid (non-start-point) candidates. + let mut valid_indices: Vec = (0..rc) + .filter(|&ri| neighbors[ri].id != START_POINT_ID) + .collect(); + + // Rerank: read full vector and compute exact distance for each candidate. + if let Ok(mut renv) = rerank_ctx.jvm.attach_current_thread() { + for &ri in &valid_indices { + let id = neighbors[ri].id; + let position = (id as i64) - 1; + let success = renv.call_method( + &rerank_ctx.reader_ref, + "loadVector", + "(J)Z", + &[jni::objects::JValue::Long(position)], + ); + if let Ok(v) = success { + if let Ok(true) = v.z() { + let vec = unsafe { + std::slice::from_raw_parts(rerank_ctx.single_buf_ptr, rerank_ctx.dim) + }; + neighbors[ri].distance = + compute_exact_distance(qvec, vec, rerank_ctx.metric_type); + } + } + } + + // Re-sort valid candidates by exact distance. + valid_indices.sort_by(|&a, &b| { + neighbors[a] + .distance + .partial_cmp(&neighbors[b].distance) + .unwrap_or(std::cmp::Ordering::Equal) + }); + + // Collect results from the re-sorted order. + let mut cnt = 0; + for &ri in &valid_indices { + if cnt >= top_k { break; } + let nb = &neighbors[ri]; + let idx = qi * top_k + cnt; + result_lbl[idx] = state.min_ext_id + (nb.id as i64) - 1; + result_dist[idx] = nb.distance; + cnt += 1; + } + } + } else { + // No PQ / no reranking — distances are already exact. + let mut cnt = 0; + for ri in 0..rc { + if cnt >= top_k { break; } + let nb = &neighbors[ri]; + if nb.id == START_POINT_ID { continue; } + let idx = qi * top_k + cnt; + result_lbl[idx] = state.min_ext_id + (nb.id as i64) - 1; + result_dist[idx] = nb.distance; + cnt += 1; + } } } diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs index 00b9dcd81b0b..b51eaaaaaab0 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs @@ -81,6 +81,115 @@ impl From for ANNError { diskann::always_escalate!(FileIOProviderError); +// ======================== PQ State ======================== + +/// In-memory Product Quantization state for approximate distance computation. +/// +/// During beam search, PQ-reconstructed vectors replace full-precision disk I/O, +/// making the search almost entirely in-memory. Only the final top-K candidates +/// are re-ranked with full-precision vectors from disk. +#[derive(Debug)] +pub struct PQState { + /// Number of PQ subspaces (M). + pub num_subspaces: usize, + /// Number of centroids per subspace (K). + pub num_centroids: usize, + /// Sub-vector dimension (dimension / M). + pub sub_dim: usize, + /// Full vector dimension. + pub dimension: usize, + /// Centroid data, laid out as: pivots[m * K * sub_dim + k * sub_dim .. + sub_dim]. + pub pivots: Vec, + /// Compressed codes: codes[vec_idx * M + m] = centroid index for vector vec_idx, subspace m. + pub codes: Vec, + /// Number of encoded vectors. + pub num_vectors: usize, +} + +impl PQState { + /// Deserialize PQ pivots and compressed codes from the byte arrays written by `pq.rs`. + /// + /// Pivots format: i32 dim | i32 M | i32 K | i32 sub_dim | f32[M*K*sub_dim] + /// Codes format: i32 N | i32 M | byte[N*M] + pub fn deserialize(pivots_bytes: &[u8], compressed_bytes: &[u8]) -> Result { + if pivots_bytes.len() < 16 { + return Err("PQ pivots too small".into()); + } + if compressed_bytes.len() < 8 { + return Err("PQ compressed too small".into()); + } + + let dimension = i32::from_ne_bytes(pivots_bytes[0..4].try_into().unwrap()) as usize; + let num_subspaces = i32::from_ne_bytes(pivots_bytes[4..8].try_into().unwrap()) as usize; + let num_centroids = i32::from_ne_bytes(pivots_bytes[8..12].try_into().unwrap()) as usize; + let sub_dim = i32::from_ne_bytes(pivots_bytes[12..16].try_into().unwrap()) as usize; + + let expected_pivots_data = num_subspaces * num_centroids * sub_dim * 4; + if pivots_bytes.len() < 16 + expected_pivots_data { + return Err(format!( + "PQ pivots data too small: need {}, have {}", + 16 + expected_pivots_data, + pivots_bytes.len() + )); + } + + // Parse pivots as f32 (native endian). + let pivots: Vec = pivots_bytes[16..16 + expected_pivots_data] + .chunks_exact(4) + .map(|c| f32::from_ne_bytes(c.try_into().unwrap())) + .collect(); + + // Parse compressed header. + let num_vectors = i32::from_ne_bytes(compressed_bytes[0..4].try_into().unwrap()) as usize; + let m_check = i32::from_ne_bytes(compressed_bytes[4..8].try_into().unwrap()) as usize; + if m_check != num_subspaces { + return Err(format!( + "PQ subspace mismatch: pivots M={}, compressed M={}", + num_subspaces, m_check + )); + } + + let expected_codes = num_vectors * num_subspaces; + if compressed_bytes.len() < 8 + expected_codes { + return Err(format!( + "PQ codes too small: need {}, have {}", + 8 + expected_codes, + compressed_bytes.len() + )); + } + + let codes = compressed_bytes[8..8 + expected_codes].to_vec(); + + Ok(Self { + num_subspaces, + num_centroids, + sub_dim, + dimension, + pivots, + codes, + num_vectors, + }) + } + + /// Reconstruct an approximate vector for the given 0-based vector index + /// by looking up PQ centroid sub-vectors. + /// + /// Cost: M table lookups + dimension float copies — entirely in L1/L2 cache. + #[inline] + pub fn reconstruct(&self, vec_idx: usize, out: &mut [f32]) { + debug_assert!(vec_idx < self.num_vectors); + debug_assert!(out.len() >= self.dimension); + let code_base = vec_idx * self.num_subspaces; + for m in 0..self.num_subspaces { + let code = self.codes[code_base + m] as usize; + let src_offset = m * self.num_centroids * self.sub_dim + code * self.sub_dim; + let dst_offset = m * self.sub_dim; + out[dst_offset..dst_offset + self.sub_dim] + .copy_from_slice(&self.pivots[src_offset..src_offset + self.sub_dim]); + } + } +} + // ======================== Graph Term ======================== /// One entry in the graph cache: its neighbor list. @@ -95,15 +204,18 @@ pub struct GraphTerm { /// Graph neighbors and vectors are read on demand from FileIO-backed storage /// (local, HDFS, S3, OSS, etc.) via JNI callbacks to Java reader objects. /// -/// Two levels of caching reduce FileIO/JNI round-trips: -/// - **Graph**: `DashMap` (lazy, write-once, unbounded). -/// - **Vectors**: `DashMap>` populated by batch prefetch -/// after each neighbor expansion, plus per-search LRU in `FileIOAccessor`. +/// Three levels of vector access (in priority order): +/// 1. **PQ reconstruction** (in-memory, ~O(dim) CPU, no I/O) — used during +/// beam search when PQ data is available. +/// 2. **Provider-level `vector_cache`** (exact vectors cached from reranking +/// or disk I/O). +/// 3. **DirectByteBuffer disk I/O** (single JNI call + zero-copy read). +/// +/// Graph neighbors are cached in a `DashMap` (lazy, write-once). pub struct FileIOProvider { /// Graph cache: internal_id → { neighbors }. graph: DashMap, - /// Provider-level vector cache populated by batch prefetch. - /// Keyed by internal node ID (u32). + /// Provider-level vector cache (exact vectors from reranking / disk reads). vector_cache: DashMap>, /// Total number of nodes (start point + user vectors). num_nodes: usize, @@ -122,13 +234,15 @@ pub struct FileIOProvider { /// Max degree. max_degree: usize, /// Native memory address of the single-vector DirectByteBuffer. - /// Points to `dim` floats. Valid for the lifetime of the Java reader. single_buf_ptr: *mut f32, /// Native memory address of the batch DirectByteBuffer. - /// Points to `max_batch_size * dim` floats. batch_buf_ptr: *mut f32, /// Maximum number of vectors in one batch read. max_batch_size: usize, + /// PQ state for in-memory approximate distance computation during beam search. + /// When present, `get_element` returns PQ-reconstructed vectors for cache misses + /// instead of doing disk I/O. + pq_state: Option, } impl std::fmt::Debug for FileIOProvider { @@ -139,6 +253,7 @@ impl std::fmt::Debug for FileIOProvider { .field("max_degree", &self.max_degree) .field("graph_size", &self.graph.len()) .field("vector_cache_size", &self.vector_cache.len()) + .field("pq_enabled", &self.pq_state.is_some()) .finish() } } @@ -150,8 +265,8 @@ unsafe impl Send for FileIOProvider {} unsafe impl Sync for FileIOProvider {} impl FileIOProvider { - /// Build a search-only provider with on-demand graph reading and zero-copy - /// vector access via DirectByteBuffers. + /// Build a search-only provider with on-demand graph reading, zero-copy + /// vector access, and optional PQ for in-memory approximate search. /// /// `single_buf_ptr` and `batch_buf_ptr` are native addresses obtained via /// JNI `GetDirectBufferAddress` on the Java reader's DirectByteBuffers. @@ -169,6 +284,7 @@ impl FileIOProvider { single_buf_ptr: *mut f32, batch_buf_ptr: *mut f32, max_batch_size: usize, + pq_state: Option, ) -> Self { let graph = DashMap::new(); let vector_cache = DashMap::new(); @@ -190,9 +306,15 @@ impl FileIOProvider { single_buf_ptr, batch_buf_ptr, max_batch_size, + pq_state, } } + /// Whether PQ is available for in-memory approximate distance computation. + pub fn has_pq(&self) -> bool { + self.pq_state.is_some() + } + pub fn dim(&self) -> usize { self.dim } @@ -493,10 +615,15 @@ impl provider::NeighborAccessor for FileIONeighborAccessor<'_> { impl FileIONeighborAccessor<'_> { /// Batch-prefetch vectors for neighbors that aren't already in the vector cache. /// - /// This is the key optimization: after discovering neighbors via graph I/O, - /// we batch-fetch all their vectors in a single JNI call. Subsequent - /// `get_element` calls for these neighbors will hit the provider-level cache. + /// When PQ is available, this is a no-op — beam search uses PQ-reconstructed + /// vectors (in-memory) instead of full-precision disk I/O. Only when PQ is + /// NOT available do we batch-fetch full vectors from disk. fn prefetch_neighbor_vectors(&self, adj: &AdjacencyList) { + // When PQ is available, beam search uses PQ reconstruction — skip disk prefetch. + if self.provider.has_pq() { + return; + } + // Collect uncached neighbor IDs (skip start points and already-cached). let uncached: Vec = adj .iter() @@ -594,26 +721,35 @@ impl provider::Accessor for FileIOAccessor<'_> { return Ok(&*self.buffer); } - // 2. Provider-level vector cache (populated by batch prefetch). + // 2. Provider-level vector cache (exact vectors from reranking / disk). if let Some(cached) = self.provider.vector_cache.get(&id) { self.buffer.copy_from_slice(&cached); return Ok(&*self.buffer); } - // 3. Per-search LRU cache. + // 3. Per-search LRU cache (exact vectors). if let Some(cached) = self.cache.get(id) { self.buffer.copy_from_slice(cached); return Ok(&*self.buffer); } - // 4. Fetch via DirectByteBuffer I/O (single JNI call + zero-copy read). + // 4. PQ reconstruction: approximate vector, entirely in-memory, no disk I/O. + // During beam search this is the primary hot path when PQ is available. + if let Some(pq) = &self.provider.pq_state { + let vec_idx = (id as usize).wrapping_sub(1); + if vec_idx < pq.num_vectors { + pq.reconstruct(vec_idx, &mut self.buffer); + return Ok(&*self.buffer); + } + } + + // 5. Fallback: fetch via DirectByteBuffer I/O (single JNI call + zero-copy read). let position = (id as i64) - 1; let fetched = self.provider.fetch_vector(position)?; match fetched { Some(vec) if vec.len() == self.provider.dim() => { self.buffer.copy_from_slice(&vec); - // Populate both caches. self.provider .vector_cache .insert(id, vec.clone().into_boxed_slice()); From 8995fcfc5e12708dc995ee940547579640bb4c53 Mon Sep 17 00:00:00 2001 From: yantian Date: Sat, 14 Feb 2026 17:11:09 +0800 Subject: [PATCH 7/7] fix diskann search flow --- .../spark/sql/DiskAnnVectorIndexE2ETest.scala | 11 +- .../index/DiskAnnVectorGlobalIndexReader.java | 27 +- .../src/main/native/src/lib.rs | 416 ++++++++++-------- .../main/native/src/paimon_fileio_provider.rs | 173 ++++++-- 4 files changed, 409 insertions(+), 218 deletions(-) diff --git a/paimon-diskann/paimon-diskann-e2e-test/src/test/scala/org/apache/paimon/spark/sql/DiskAnnVectorIndexE2ETest.scala b/paimon-diskann/paimon-diskann-e2e-test/src/test/scala/org/apache/paimon/spark/sql/DiskAnnVectorIndexE2ETest.scala index 1ba4dd7144de..e75ed462323e 100644 --- a/paimon-diskann/paimon-diskann-e2e-test/src/test/scala/org/apache/paimon/spark/sql/DiskAnnVectorIndexE2ETest.scala +++ b/paimon-diskann/paimon-diskann-e2e-test/src/test/scala/org/apache/paimon/spark/sql/DiskAnnVectorIndexE2ETest.scala @@ -367,7 +367,7 @@ class DiskAnnVectorIndexE2ETest extends PaimonSparkTestBase { .filter(_.indexFile().indexType() == "diskann-vector-ann") assert(indexEntries.nonEmpty) - val searchResult = spark + var searchResult = spark .sql( """ |SELECT id, name FROM vector_search('T', 'embedding', array(500.0f, 501.0f, 502.0f), 10) @@ -375,6 +375,15 @@ class DiskAnnVectorIndexE2ETest extends PaimonSparkTestBase { .collect() assert(searchResult.exists(row => row.getInt(0) == 500 && row.getString(1) == "item_500")) + + searchResult = spark + .sql( + """ + |SELECT id, name FROM vector_search('T', 'embedding', array(501.0f, 502.0f, 503.0f), 10) + |""".stripMargin) + .collect() + + assert(searchResult.exists(row => row.getInt(0) == 501 && row.getString(1) == "item_501")) } } } diff --git a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java index 184e442e0240..44ad2955594a 100644 --- a/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java +++ b/paimon-diskann/paimon-diskann-index/src/main/java/org/apache/paimon/diskann/index/DiskAnnVectorGlobalIndexReader.java @@ -336,10 +336,26 @@ private void loadIndexAt(int position) throws IOException { new FileIOVectorReader(vectorStream, meta.dim(), meta.maxDegree()); // 3. Load PQ files into memory for in-memory approximate distance computation. - // Beam search uses PQ-reconstructed vectors; only top-K candidates are - // re-ranked with full-precision vectors from disk. + // PQ is mandatory — beam search uses PQ brute-force scan followed by + // full-precision reranking from disk. byte[] pqPivots = loadCompanionFile(ioMeta, meta.pqPivotsFileName()); byte[] pqCompressed = loadCompanionFile(ioMeta, meta.pqCompressedFileName()); + if (pqPivots == null || pqPivots.length == 0) { + throw new IOException( + "PQ pivots file is missing or empty for index at position " + + position + + ". PQ is required for DiskANN search. " + + "Pivots file: " + + meta.pqPivotsFileName()); + } + if (pqCompressed == null || pqCompressed.length == 0) { + throw new IOException( + "PQ compressed file is missing or empty for index at position " + + position + + ". PQ is required for DiskANN search. " + + "Compressed file: " + + meta.pqCompressedFileName()); + } // 4. Create DiskANN native searcher with on-demand graph + vector access + PQ. handle = @@ -402,8 +418,8 @@ public void close() throws IOException { /** * Load a companion file (e.g. PQ pivots/compressed) relative to the index file. * - * @return the file contents as byte[], or null if the file name is empty or the file does not - * exist. + * @return the file contents as byte[], or null if the file name is null/empty. + * @throws IOException if the file cannot be read. */ private byte[] loadCompanionFile(GlobalIndexIOMeta indexIOMeta, String fileName) throws IOException { @@ -422,9 +438,6 @@ private byte[] loadCompanionFile(GlobalIndexIOMeta indexIOMeta, String fileName) } byte[] data = baos.toByteArray(); return data.length > 0 ? data : null; - } catch (Exception e) { - // PQ files are optional — if missing, fall back to full-precision search. - return null; } } diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs index 332cb9ff519e..27e04f72f101 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs @@ -45,7 +45,7 @@ use diskann_vector::distance::Metric; mod paimon_fileio_provider; mod pq; -use paimon_fileio_provider::{FileIOContext, FileIOProvider, FileIOStrategy}; +use paimon_fileio_provider::FileIOProvider; // ======================== Constants ======================== @@ -578,27 +578,26 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearch< // ---- Searcher registry (handles backed by FileIOProvider) ---- struct SearcherState { + /// The DiskANN index (holds the FileIOProvider which has PQ data + graph/vector readers). index: Arc>, - context: FileIOContext, - runtime: tokio::runtime::Runtime, dimension: i32, - /// Minimum external ID for this index. ext_id = min_ext_id + (int_id - 1). + /// Minimum external ID for this index. ext_id = min_ext_id + vec_idx (0-based). min_ext_id: i64, - /// Reranking context: used after PQ-approximate beam search to compute - /// exact distances for the top-K candidates with full-precision vectors. - rerank: Option, + /// Graph start/entry point (medoid) internal ID. + start_id: u32, + /// I/O context for beam search: JVM, reader refs, DirectByteBuffer pointers. + io_ctx: BeamSearchIOContext, } -/// Context for post-search reranking with full-precision vectors from disk. -/// -/// When PQ is enabled, beam search uses PQ-approximate distances (in-memory). -/// After search, we re-read full vectors for the top-K candidates and recompute -/// exact distances to produce the final ranking. -struct RerankContext { - /// Separate JVM handle for reranking (cheap clone of the pointer). +/// I/O context for beam search. Provides JNI access to graph reader (for +/// neighbor lists) and vector reader (for full-precision vectors on disk). +struct BeamSearchIOContext { + /// JVM handle for attaching threads. jvm: jni::JavaVM, - /// Separate GlobalRef to the vector reader (same Java object, different ref). - reader_ref: jni::objects::GlobalRef, + /// GlobalRef to the Java vector reader (`FileIOVectorReader`). + vector_reader_ref: jni::objects::GlobalRef, + /// GlobalRef to the Java graph reader (`FileIOGraphReader`). + graph_reader_ref: jni::objects::GlobalRef, /// Native address of the single-vector DirectByteBuffer. single_buf_ptr: *mut f32, /// Vector dimension. @@ -609,8 +608,8 @@ struct RerankContext { // SAFETY: same justification as FileIOProvider — JavaVM and GlobalRef are // Send+Sync, raw pointer access is serialized by single-threaded runtime. -unsafe impl Send for RerankContext {} -unsafe impl Sync for RerankContext {} +unsafe impl Send for BeamSearchIOContext {} +unsafe impl Sync for BeamSearchIOContext {} /// Compute exact distance between two vectors. fn compute_exact_distance(a: &[f32], b: &[f32], metric_type: i32) -> f32 { @@ -786,14 +785,14 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSeriali // ======================== indexCreateSearcherFromReaders ======================== /// Create a search-only handle from two on-demand Java readers: one for graph -/// structure and one for vectors, with optional PQ data for in-memory -/// approximate distance computation during beam search. +/// structure and one for vectors, plus PQ data for in-memory approximate +/// distance computation during beam search. /// /// `graphReader`: Java object with `readNeighbors(int)`, `getDimension()`, etc. /// `vectorReader`: Java object with `loadVector(long)`, DirectByteBuffer accessors. /// `min_ext_id`: Minimum external ID for int_id → ext_id conversion. -/// `pq_pivots`: Serialized PQ codebook (byte[]), or null to disable PQ. -/// `pq_compressed`: Serialized PQ compressed codes (byte[]), or null. +/// `pq_pivots`: Serialized PQ codebook (byte[]). Must not be null. +/// `pq_compressed`: Serialized PQ compressed codes (byte[]). Must not be null. /// /// Returns a searcher handle (≥100000) for use with `indexSearchWithReader`. #[no_mangle] @@ -873,60 +872,55 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS Err(_) => max_degree, }; - // ---- Deserialize PQ data (if provided) ---- + // ---- Deserialize PQ data (always required — Java has validated this) ---- - let pq_state = if !pq_pivots.is_null() && !pq_compressed.is_null() { - let pivots_bytes: Vec = match env.convert_byte_array( - jni::objects::JByteArray::from(pq_pivots), - ) { - Ok(b) if !b.is_empty() => b, - _ => Vec::new(), - }; - let compressed_bytes: Vec = match env.convert_byte_array( - jni::objects::JByteArray::from(pq_compressed), - ) { - Ok(b) if !b.is_empty() => b, - _ => Vec::new(), - }; + if pq_pivots.is_null() || pq_compressed.is_null() { + let _ = env.throw_new("java/lang/IllegalArgumentException", "PQ pivots and compressed data must not be null"); + return 0; + } - if !pivots_bytes.is_empty() && !compressed_bytes.is_empty() { - match paimon_fileio_provider::PQState::deserialize(&pivots_bytes, &compressed_bytes) { - Ok(pq) => Some(pq), - Err(e) => { - // PQ deserialization failed — fall back to full-precision search. - eprintln!("PQ deserialization failed (falling back to full search): {}", e); - None - } - } - } else { - None - } - } else { - None + let pivots_bytes: Vec = match env.convert_byte_array( + jni::objects::JByteArray::from(pq_pivots), + ) { + Ok(b) if !b.is_empty() => b, + _ => { let _ = env.throw_new("java/lang/IllegalArgumentException", "PQ pivots byte array is empty"); return 0; } + }; + let compressed_bytes: Vec = match env.convert_byte_array( + jni::objects::JByteArray::from(pq_compressed), + ) { + Ok(b) if !b.is_empty() => b, + _ => { let _ = env.throw_new("java/lang/IllegalArgumentException", "PQ compressed byte array is empty"); return 0; } }; - let pq_enabled = pq_state.is_some(); + let pq_state = match paimon_fileio_provider::PQState::deserialize(&pivots_bytes, &compressed_bytes) { + Ok(pq) => pq, + Err(e) => { + let _ = env.throw_new("java/lang/RuntimeException", format!("PQ deserialization failed: {}", e)); + return 0; + } + }; - // ---- Create reranking context (separate GlobalRef + JVM for post-search reranking) ---- + // ---- Create beam search I/O context (separate GlobalRefs + JVM for search I/O) ---- - let rerank = if pq_enabled { - let rerank_reader_ref = match env.new_global_ref(&vector_reader) { - Ok(g) => g, - Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("rerank reader ref: {}", e)); return 0; } - }; - let rerank_jvm = match env.get_java_vm() { - Ok(vm) => vm, - Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("rerank JVM: {}", e)); return 0; } - }; - Some(RerankContext { - jvm: rerank_jvm, - reader_ref: rerank_reader_ref, - single_buf_ptr, - dim, - metric_type, - }) - } else { - None + let io_vector_ref = match env.new_global_ref(&vector_reader) { + Ok(g) => g, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("io vector ref: {}", e)); return 0; } + }; + let io_graph_ref = match env.new_global_ref(&graph_reader) { + Ok(g) => g, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("io graph ref: {}", e)); return 0; } + }; + let io_jvm = match env.get_java_vm() { + Ok(vm) => vm, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("io JVM: {}", e)); return 0; } + }; + let io_ctx = BeamSearchIOContext { + jvm: io_jvm, + vector_reader_ref: io_vector_ref, + graph_reader_ref: io_graph_ref, + single_buf_ptr, + dim, + metric_type, }; // Start point is not stored in data file; use a dummy vector. @@ -949,7 +943,7 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS pq_state, ); - // Build DiskANNIndex config. + // Build DiskANNIndex config (still needed for the provider wrapper). let md = std::cmp::max(max_degree, 4); let bls = std::cmp::max(build_ls, md); let metric = map_metric(metric_type); @@ -964,23 +958,14 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("config: {:?}", e)); return 0; } }; - let runtime = match tokio::runtime::Builder::new_current_thread() - .enable_all() - .build() - { - Ok(r) => r, - Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("runtime: {}", e)); return 0; } - }; - let index = Arc::new(DiskANNIndex::new(index_config, provider, None)); let searcher = SearcherState { index, - context: FileIOContext, - runtime, dimension, min_ext_id, - rerank, + start_id, + io_ctx, }; match searcher_registry().lock() { @@ -992,6 +977,18 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateS // ======================== indexSearchWithReader ======================== /// Search on a searcher handle created by `indexCreateSearcherFromReaders`. +/// +/// Implements the standard DiskANN search algorithm: +/// +/// 1. **Start** from the medoid (graph entry point). +/// 2. **Beam Search loop**: +/// - Pop the unvisited node with the smallest **PQ distance** from the beam. +/// - **Disk I/O**: read its full vector + neighbor list via JNI. +/// - **Compute Exact**: use the full vector to compute exact distance, update +/// the result heap. +/// - **Expand**: for each neighbor, compute **PQ distance** (in-memory). +/// - **Push**: add neighbors with good PQ distance to the beam (capped at L). +/// 3. **Return** the result heap (already exactly sorted). #[no_mangle] pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearchWithReader<'local>( mut env: JNIEnv<'local>, @@ -1034,116 +1031,189 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearchW let mut result_dist = vec![f32::MAX; total]; let mut result_lbl = vec![-1i64; total]; - let strat = FileIOStrategy::new(); + let provider = state.index.provider(); + let pq = &provider.pq_state; + let io = &state.io_ctx; + let start_id = state.start_id; + + // Attach JNI thread once for all queries. + let mut jni_env = match io.jvm.attach_current_thread() { + Ok(e) => e, + Err(e) => { + let _ = env.throw_new("java/lang/RuntimeException", format!("JVM attach: {}", e)); + return; + } + }; for qi in 0..num { let qvec = &query[qi * dimension..(qi + 1) * dimension]; - let search_k = top_k + 1; - let l = std::cmp::max(search_list_size as usize, search_k); - - let params = match graph::SearchParams::new(search_k, l, None) { - Ok(p) => p, - Err(e) => { - let _ = env.throw_new("java/lang/IllegalArgumentException", - format!("Invalid search params: {}", e)); - return; - } - }; + let l = std::cmp::max(search_list_size as usize, top_k); - let mut neighbors = vec![Neighbor::::default(); search_k]; + // ---- Pre-compute PQ distance lookup table for this query ---- + let distance_table = pq.compute_distance_table(qvec, io.metric_type); - let idx_clone = Arc::clone(&state.index); - let search_result = panic::catch_unwind(AssertUnwindSafe(|| { - state.runtime.block_on(idx_clone.search( - &strat, - &state.context, - qvec, - ¶ms, - &mut BackInserter::new(&mut neighbors), - )) - })); + // ---- Beam: sorted candidate list, capped at L entries ---- + // Each entry: (pq_distance, internal_node_id). + // Sorted ascending by pq_distance so beam[0] is always the closest. + let mut beam: Vec<(f32, u32)> = Vec::with_capacity(l + 1); + let mut visited = std::collections::HashSet::::with_capacity(l * 2); - let stats = match search_result { - Ok(Ok(s)) => s, - Ok(Err(e)) => { - let _ = env.throw_new("java/lang/RuntimeException", - format!("Search failed: {}", e)); - return; - } - Err(_) => { - let _ = env.throw_new("java/lang/RuntimeException", "Search panicked"); - return; + // ---- Result heap: max-heap of (exact_distance, vec_idx) capped at top_k ---- + let mut results: Vec<(f32, usize)> = Vec::with_capacity(top_k + 1); + let mut result_worst = f32::MAX; + + // Seed beam with the start point (medoid). + // start_id is an internal node ID; vec_idx = start_id - 1. + { + let start_vec_idx = (start_id as usize).wrapping_sub(1); + let start_pq_dist = if start_vec_idx < pq.num_vectors { + pq.adc_distance(start_vec_idx, &distance_table, io.metric_type) + } else { + f32::MAX + }; + beam.push((start_pq_dist, start_id)); + } + + // ---- Beam search loop ---- + loop { + // Find the closest unvisited candidate in the beam. + let next = beam.iter() + .enumerate() + .filter(|(_, (_, id))| !visited.contains(id)) + .min_by(|a, b| a.1.0.partial_cmp(&b.1.0).unwrap_or(std::cmp::Ordering::Equal)) + .map(|(idx, &(dist, id))| (idx, dist, id)); + + let (_beam_idx, _pq_dist, node_id) = match next { + Some(t) => t, + None => break, // No more unvisited candidates — convergence. + }; + + visited.insert(node_id); + + // ---- Disk I/O: read full vector for this node ---- + // (skip start point if it has no data vector — int_id 0 is synthetic) + let vec_idx = (node_id as usize).wrapping_sub(1); + + if node_id != START_POINT_ID && vec_idx < pq.num_vectors { + let position = vec_idx as i64; + let load_ok = jni_env.call_method( + &io.vector_reader_ref, + "loadVector", + "(J)Z", + &[jni::objects::JValue::Long(position)], + ); + if let Ok(v) = load_ok { + if let Ok(true) = v.z() { + let full_vec = unsafe { + std::slice::from_raw_parts(io.single_buf_ptr, io.dim) + }; + let exact_dist = compute_exact_distance(qvec, full_vec, io.metric_type); + + // Update result heap (keep top_k smallest exact distances). + if results.len() < top_k { + results.push((exact_dist, vec_idx)); + if results.len() == top_k { + result_worst = results.iter() + .map(|e| e.0) + .fold(f32::NEG_INFINITY, f32::max); + } + } else if exact_dist < result_worst { + // Replace the worst entry. + if let Some(pos) = results.iter().position(|e| e.0 == result_worst) { + results[pos] = (exact_dist, vec_idx); + result_worst = results.iter() + .map(|e| e.0) + .fold(f32::NEG_INFINITY, f32::max); + } + } + } + } } - }; - let rc = stats.result_count as usize; - - // ---- Reranking: replace PQ-approximate distances with exact distances ---- - // When PQ is enabled, beam search used PQ-reconstructed vectors for distance - // computation. Now we read full-precision vectors from disk for the top - // candidates and recompute exact distances, then re-sort. - if let Some(ref rerank_ctx) = state.rerank { - // Collect valid (non-start-point) candidates. - let mut valid_indices: Vec = (0..rc) - .filter(|&ri| neighbors[ri].id != START_POINT_ID) - .collect(); - - // Rerank: read full vector and compute exact distance for each candidate. - if let Ok(mut renv) = rerank_ctx.jvm.attach_current_thread() { - for &ri in &valid_indices { - let id = neighbors[ri].id; - let position = (id as i64) - 1; - let success = renv.call_method( - &rerank_ctx.reader_ref, - "loadVector", - "(J)Z", - &[jni::objects::JValue::Long(position)], + // ---- Read neighbor list for this node ---- + let neighbors: Vec = { + // Try graph cache first. + if let Some(term) = provider.graph.get(&node_id) { + term.neighbors.iter().copied().collect() + } else { + // Fetch from graph reader via JNI. + let ret = jni_env.call_method( + &io.graph_reader_ref, + "readNeighbors", + "(I)[I", + &[jni::objects::JValue::Int(node_id as i32)], ); - if let Ok(v) = success { - if let Ok(true) = v.z() { - let vec = unsafe { - std::slice::from_raw_parts(rerank_ctx.single_buf_ptr, rerank_ctx.dim) - }; - neighbors[ri].distance = - compute_exact_distance(qvec, vec, rerank_ctx.metric_type); + match ret { + Ok(v) => match v.l() { + Ok(obj) if !obj.is_null() => { + let int_array = jni::objects::JIntArray::from(obj); + let arr_len = jni_env.get_array_length(&int_array).unwrap_or(0) as usize; + let mut buf = vec![0i32; arr_len]; + let _ = jni_env.get_int_array_region(&int_array, 0, &mut buf); + let nbrs: Vec = buf.into_iter().map(|v| v as u32).collect(); + // Cache for future queries. + let adj = diskann::graph::AdjacencyList::from_iter_untrusted(nbrs.iter().copied()); + provider.graph.insert(node_id, paimon_fileio_provider::GraphTerm { neighbors: adj }); + nbrs + } + _ => Vec::new(), + }, + Err(_) => { + let _ = jni_env.exception_clear(); + Vec::new() } } } + }; + + // ---- Expand: compute PQ distance for each neighbor, add to beam ---- + let beam_worst = if beam.len() >= l { + beam.last().map(|e| e.0).unwrap_or(f32::MAX) + } else { + f32::MAX + }; + + for &nbr_id in &neighbors { + if visited.contains(&nbr_id) { + continue; + } + // Already in beam? Skip duplicate. + if beam.iter().any(|&(_, id)| id == nbr_id) { + continue; + } + + let nbr_vec_idx = (nbr_id as usize).wrapping_sub(1); + let nbr_pq_dist = if nbr_vec_idx < pq.num_vectors { + pq.adc_distance(nbr_vec_idx, &distance_table, io.metric_type) + } else { + f32::MAX + }; - // Re-sort valid candidates by exact distance. - valid_indices.sort_by(|&a, &b| { - neighbors[a] - .distance - .partial_cmp(&neighbors[b].distance) - .unwrap_or(std::cmp::Ordering::Equal) - }); - - // Collect results from the re-sorted order. - let mut cnt = 0; - for &ri in &valid_indices { - if cnt >= top_k { break; } - let nb = &neighbors[ri]; - let idx = qi * top_k + cnt; - result_lbl[idx] = state.min_ext_id + (nb.id as i64) - 1; - result_dist[idx] = nb.distance; - cnt += 1; + if beam.len() < l || nbr_pq_dist < beam_worst { + // Insert in sorted order. + let insert_pos = beam.partition_point(|e| e.0 < nbr_pq_dist); + beam.insert(insert_pos, (nbr_pq_dist, nbr_id)); + // Trim to L entries. + if beam.len() > l { + beam.truncate(l); + } } } - } else { - // No PQ / no reranking — distances are already exact. - let mut cnt = 0; - for ri in 0..rc { - if cnt >= top_k { break; } - let nb = &neighbors[ri]; - if nb.id == START_POINT_ID { continue; } - let idx = qi * top_k + cnt; - result_lbl[idx] = state.min_ext_id + (nb.id as i64) - 1; - result_dist[idx] = nb.distance; - cnt += 1; + } + + // ---- Collect top-K results (sorted by exact distance) ---- + results.sort_by(|a, b| a.0.partial_cmp(&b.0).unwrap_or(std::cmp::Ordering::Equal)); + for (cnt, &(exact_dist, vec_idx)) in results.iter().enumerate() { + if cnt >= top_k { + break; } + let idx = qi * top_k + cnt; + result_lbl[idx] = state.min_ext_id + vec_idx as i64; + result_dist[idx] = exact_dist; } } + drop(jni_env); drop(state); // Write back distances. diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs index b51eaaaaaab0..149a9d83ae69 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/paimon_fileio_provider.rs @@ -188,6 +188,112 @@ impl PQState { .copy_from_slice(&self.pivots[src_offset..src_offset + self.sub_dim]); } } + + // ---- ADC (Asymmetric Distance Computation) for brute-force PQ search ---- + + /// Pre-compute a distance table from a query vector to all PQ centroids. + /// + /// Returns `dt[m * K + k]` where: + /// - L2: squared L2 distance between query sub-vector m and centroid (m, k) + /// - IP/Cosine: dot product between query sub-vector m and centroid (m, k) + /// + /// Cost: O(M * K * sub_dim) — computed once per query, amortized over all vectors. + pub fn compute_distance_table(&self, query: &[f32], metric_type: i32) -> Vec { + let m = self.num_subspaces; + let k = self.num_centroids; + let sd = self.sub_dim; + let mut table = vec![0.0f32; m * k]; + for mi in 0..m { + let q_start = mi * sd; + let q_sub = &query[q_start..q_start + sd]; + for ki in 0..k { + let c_off = mi * k * sd + ki * sd; + let centroid = &self.pivots[c_off..c_off + sd]; + let val = if metric_type == 1 || metric_type == 2 { + // IP or Cosine: dot product per subspace. + q_sub.iter().zip(centroid).map(|(a, b)| a * b).sum::() + } else { + // L2: squared L2 per subspace. + q_sub + .iter() + .zip(centroid) + .map(|(a, b)| { + let d = a - b; + d * d + }) + .sum::() + }; + table[mi * k + ki] = val; + } + } + table + } + + /// Compute the approximate PQ distance for one vector using a pre-computed + /// distance table. Cost: O(M) — just M table lookups. + #[inline] + pub fn adc_distance(&self, vec_idx: usize, table: &[f32], metric_type: i32) -> f32 { + let base = vec_idx * self.num_subspaces; + let k = self.num_centroids; + let mut raw = 0.0f32; + for mi in 0..self.num_subspaces { + let code = self.codes[base + mi] as usize; + raw += table[mi * k + code]; + } + // IP/Cosine: negate dot product so that larger similarity = smaller distance. + if metric_type == 1 || metric_type == 2 { + -raw + } else { + raw + } + } + + /// Brute-force PQ search: scan all vectors with ADC, return the top-K + /// closest as `(vec_idx, pq_distance)` sorted by ascending distance. + /// + /// This replaces the graph-based beam search when PQ data is available. + /// The entire search is in-memory — zero graph I/O, zero vector disk I/O. + /// + /// Cost: O(M * K * sub_dim) for the distance table + + /// O(N * M) for scanning all vectors. + pub fn brute_force_search( + &self, + query: &[f32], + top_k: usize, + metric_type: i32, + ) -> Vec<(usize, f32)> { + let table = self.compute_distance_table(query, metric_type); + let k = top_k.min(self.num_vectors); + if k == 0 { + return Vec::new(); + } + + // Max-heap to keep the top-k smallest distances. + // We store (distance, vec_idx) and the heap evicts the largest distance. + let mut heap: Vec<(f32, usize)> = Vec::with_capacity(k); + let mut heap_max = f32::MAX; + + for i in 0..self.num_vectors { + let dist = self.adc_distance(i, &table, metric_type); + if heap.len() < k { + heap.push((dist, i)); + if heap.len() == k { + // Find the current maximum after filling the heap. + heap_max = heap.iter().map(|e| e.0).fold(f32::NEG_INFINITY, f32::max); + } + } else if dist < heap_max { + // Replace the worst element. + if let Some(pos) = heap.iter().position(|e| e.0 == heap_max) { + heap[pos] = (dist, i); + heap_max = heap.iter().map(|e| e.0).fold(f32::NEG_INFINITY, f32::max); + } + } + } + + // Sort by ascending distance. + heap.sort_by(|a, b| a.0.partial_cmp(&b.0).unwrap_or(std::cmp::Ordering::Equal)); + heap.into_iter().map(|(d, i)| (i, d)).collect() + } } // ======================== Graph Term ======================== @@ -214,7 +320,7 @@ pub struct GraphTerm { /// Graph neighbors are cached in a `DashMap` (lazy, write-once). pub struct FileIOProvider { /// Graph cache: internal_id → { neighbors }. - graph: DashMap, + pub graph: DashMap, /// Provider-level vector cache (exact vectors from reranking / disk reads). vector_cache: DashMap>, /// Total number of nodes (start point + user vectors). @@ -240,9 +346,8 @@ pub struct FileIOProvider { /// Maximum number of vectors in one batch read. max_batch_size: usize, /// PQ state for in-memory approximate distance computation during beam search. - /// When present, `get_element` returns PQ-reconstructed vectors for cache misses - /// instead of doing disk I/O. - pq_state: Option, + /// PQ is always present — Java validates this before creating the native searcher. + pub pq_state: PQState, } impl std::fmt::Debug for FileIOProvider { @@ -253,7 +358,7 @@ impl std::fmt::Debug for FileIOProvider { .field("max_degree", &self.max_degree) .field("graph_size", &self.graph.len()) .field("vector_cache_size", &self.vector_cache.len()) - .field("pq_enabled", &self.pq_state.is_some()) + .field("pq_enabled", &true) .finish() } } @@ -266,10 +371,11 @@ unsafe impl Sync for FileIOProvider {} impl FileIOProvider { /// Build a search-only provider with on-demand graph reading, zero-copy - /// vector access, and optional PQ for in-memory approximate search. + /// vector access, and PQ for in-memory approximate search. /// /// `single_buf_ptr` and `batch_buf_ptr` are native addresses obtained via /// JNI `GetDirectBufferAddress` on the Java reader's DirectByteBuffers. + /// PQ is always required — Java validates this before creating the native searcher. #[allow(clippy::too_many_arguments)] pub fn new_with_readers( num_nodes: usize, @@ -284,7 +390,7 @@ impl FileIOProvider { single_buf_ptr: *mut f32, batch_buf_ptr: *mut f32, max_batch_size: usize, - pq_state: Option, + pq_state: PQState, ) -> Self { let graph = DashMap::new(); let vector_cache = DashMap::new(); @@ -310,9 +416,21 @@ impl FileIOProvider { } } - /// Whether PQ is available for in-memory approximate distance computation. - pub fn has_pq(&self) -> bool { - self.pq_state.is_some() + /// PQ brute-force search: scan all PQ codes, return top-K candidates. + /// + /// Returns `vec of (vec_idx, pq_distance)`. + /// `vec_idx` is 0-based (data file position); `int_id = vec_idx + 1`. + pub fn pq_brute_force_search( + &self, + query: &[f32], + top_k: usize, + ) -> Vec<(usize, f32)> { + let metric_type = match self.metric { + Metric::InnerProduct => 1i32, + Metric::Cosine => 2i32, + _ => 0i32, + }; + self.pq_state.brute_force_search(query, top_k, metric_type) } pub fn dim(&self) -> usize { @@ -615,31 +733,11 @@ impl provider::NeighborAccessor for FileIONeighborAccessor<'_> { impl FileIONeighborAccessor<'_> { /// Batch-prefetch vectors for neighbors that aren't already in the vector cache. /// - /// When PQ is available, this is a no-op — beam search uses PQ-reconstructed - /// vectors (in-memory) instead of full-precision disk I/O. Only when PQ is - /// NOT available do we batch-fetch full vectors from disk. - fn prefetch_neighbor_vectors(&self, adj: &AdjacencyList) { - // When PQ is available, beam search uses PQ reconstruction — skip disk prefetch. - if self.provider.has_pq() { - return; - } - - // Collect uncached neighbor IDs (skip start points and already-cached). - let uncached: Vec = adj - .iter() - .copied() - .filter(|&nid| { - !self.provider.start_points.contains_key(&nid) - && !self.provider.vector_cache.contains_key(&nid) - }) - .collect(); - - if uncached.is_empty() { - return; - } - - // Best-effort: don't propagate errors from prefetch. - let _ = self.provider.prefetch_vectors(&uncached); + /// PQ is always enabled — beam search uses PQ reconstruction (in-memory), + /// so disk prefetch is a no-op. This method is kept for the graph::Strategy + /// trait requirement. + fn prefetch_neighbor_vectors(&self, _adj: &AdjacencyList) { + // No-op: PQ is always enabled, beam search uses in-memory PQ codes. } } @@ -734,8 +832,9 @@ impl provider::Accessor for FileIOAccessor<'_> { } // 4. PQ reconstruction: approximate vector, entirely in-memory, no disk I/O. - // During beam search this is the primary hot path when PQ is available. - if let Some(pq) = &self.provider.pq_state { + // During beam search this is the primary hot path. + { + let pq = &self.provider.pq_state; let vec_idx = (id as usize).wrapping_sub(1); if vec_idx < pq.num_vectors { pq.reconstruct(vec_idx, &mut self.buffer);

  • Semantics are identical to {@link Index#search}. During beam search the Rust code will + * invoke the vector reader's {@code readVector} to fetch vectors it has not yet cached. + */ + public void search( + long n, + float[] queryVectors, + int k, + int searchListSize, + float[] distances, + long[] labels) { + checkNotClosed(); + if (queryVectors.length < n * dimension) { + throw new IllegalArgumentException( + "Query vectors array too small: required " + + (n * dimension) + + ", got " + + queryVectors.length); + } + if (distances.length < n * k) { + throw new IllegalArgumentException( + "Distances array too small: required " + (n * k) + ", got " + distances.length); + } + if (labels.length < n * k) { + throw new IllegalArgumentException( + "Labels array too small: required " + (n * k) + ", got " + labels.length); + } + DiskAnnNative.indexSearchWithReader( + nativeHandle, n, queryVectors, k, searchListSize, distances, labels); + } + + private void checkNotClosed() { + if (closed) { + throw new IllegalStateException("IndexSearcher has been closed"); + } + } + + @Override + public synchronized void close() { + if (!closed) { + closed = true; + if (nativeHandle != 0) { + DiskAnnNative.indexDestroySearcher(nativeHandle); + nativeHandle = 0; + } + for (Closeable c : closeables) { + try { + c.close(); + } catch (Exception e) { + // best-effort + } + } + } + } +} diff --git a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/NativeLibraryLoader.java b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/NativeLibraryLoader.java index 6369e8913fa3..2429e4b8fd57 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/NativeLibraryLoader.java +++ b/paimon-diskann/paimon-diskann-jni/src/main/java/org/apache/paimon/diskann/NativeLibraryLoader.java @@ -263,8 +263,4 @@ private static String getArchName() { throw new UnsupportedOperationException("Unsupported architecture: " + osArch); } } - - public static String getLibraryName() { - return JNI_LIBRARY_NAME; - } } diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.lock b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.lock index 170dc286eaa7..7089eddcf145 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.lock +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.lock @@ -181,6 +181,17 @@ version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "05f29059c0c2090612e8d742178b0580d2dc940c837851ad723096f87af6663e" +[[package]] +name = "futures-macro" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "futures-task" version = "0.3.31" @@ -194,9 +205,11 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9fa08315bb612088cc391249efdc3bc77536f16c91f6cf495e6fbe85b20a4a81" dependencies = [ "futures-core", + "futures-macro", "futures-task", "pin-project-lite", "pin-utils", + "slab", ] [[package]] @@ -316,8 +329,10 @@ checksum = "42f5e15c9953c5e4ccceeb2e7382a716482c34515315f7b03532b8b4e8393d2d" name = "paimon_diskann_jni" version = "0.1.0" dependencies = [ + "dashmap", "diskann", "diskann-vector", + "futures-util", "jni", "tokio", ] @@ -443,6 +458,12 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" +[[package]] +name = "slab" +version = "0.4.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a2ae44ef20feb57a68b23d846850f861394c2e02dc425a50098ae8c90267589" + [[package]] name = "smallvec" version = "1.15.1" diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml index acd48263f9b6..2cf6a39547c5 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/Cargo.toml @@ -31,3 +31,7 @@ diskann-vector = "0.45.0" # Tokio with rt feature: Required for async DiskANN operations # Uses single-threaded runtime (new_current_thread) for efficient resource usage tokio = { version = "1", features = ["rt"] } +# Futures utilities (for futures_util::future::ok in JniProvider) +futures-util = "0.3" +# Concurrent HashMap used by the JniProvider graph storage +dashmap = "6" diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/jni_provider.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/jni_provider.rs new file mode 100644 index 000000000000..d8b4723b98cb --- /dev/null +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/jni_provider.rs @@ -0,0 +1,727 @@ +/* + * 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. + */ + +//! A DiskANN [`DataProvider`] whose graph lives in memory but whose vectors +//! are fetched on demand from Java via JNI callbacks. +//! +//! This enables the classic DiskANN architecture: +//! - **In-memory**: navigational graph (adjacency lists) + start-point vectors. +//! - **On-demand**: full-precision vectors are read through +//! `FileIOVectorReader.readVector(long)` on the Java side, which can +//! be backed by Paimon FileIO (local, HDFS, S3, OSS, etc.). + +use std::collections::HashMap; + +use dashmap::DashMap; +use diskann::graph::glue; +use diskann::graph::AdjacencyList; +use diskann::provider; +use diskann::{ANNError, ANNResult}; +use diskann_vector::distance::Metric; + +use jni::objects::GlobalRef; +use jni::sys::jlong; +use jni::JavaVM; + +use crate::map_metric; + +// ======================== Error ======================== + +#[derive(Debug, Clone)] +pub enum JniProviderError { + InvalidId(u32), + JniCallFailed(String), +} + +impl std::fmt::Display for JniProviderError { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + Self::InvalidId(id) => write!(f, "invalid vector id {}", id), + Self::JniCallFailed(msg) => write!(f, "JNI callback failed: {}", msg), + } + } +} + +impl std::error::Error for JniProviderError {} + +impl From for ANNError { + #[track_caller] + fn from(e: JniProviderError) -> ANNError { + ANNError::opaque(e) + } +} + +diskann::always_escalate!(JniProviderError); + +// ======================== LRU Cache ======================== + +/// Tiny LRU cache for recently fetched vectors to reduce JNI round-trips. +struct VectorCache { + map: HashMap>, + order: Vec, + capacity: usize, +} + +impl VectorCache { + fn new(capacity: usize) -> Self { + Self { + map: HashMap::with_capacity(capacity), + order: Vec::with_capacity(capacity), + capacity, + } + } + + fn get(&self, id: u32) -> Option<&[f32]> { + self.map.get(&id).map(|v| &**v) + } + + fn put(&mut self, id: u32, vec: Box<[f32]>) { + if self.map.contains_key(&id) { + return; + } + if self.order.len() >= self.capacity { + if let Some(evicted) = self.order.first().copied() { + self.order.remove(0); + self.map.remove(&evicted); + } + } + self.order.push(id); + self.map.insert(id, vec); + } +} + +// ======================== Graph Term ======================== + +/// One entry in the in-memory graph: its neighbors and external ID. +pub struct GraphTerm { + pub neighbors: AdjacencyList, + pub ext_id: i64, +} + +// ======================== JniProvider ======================== + +/// Data provider that keeps the graph in-memory (or lazily loaded) and reads vectors via JNI. +pub struct JniProvider { + /// In-memory graph: internal_id → { neighbors, ext_id }. + /// In on-demand mode this acts as a lazy cache — entries are populated on first access. + graph: DashMap, + /// Internal→External ID mapping (separate for fast lookup). + int_to_ext: HashMap, + /// External→Internal ID mapping. + ext_to_int: HashMap, + /// Start-point IDs and their vectors (always kept in memory). + start_points: HashMap>, + /// JVM handle for attaching threads. + jvm: JavaVM, + /// Global reference to the Java vector reader object (e.g. `FileIOVectorReader`). + reader_ref: GlobalRef, + /// Global reference to the Java graph reader object (e.g. `FileIOGraphReader`). + /// When set, graph neighbors are fetched on demand via JNI callbacks. + graph_reader_ref: Option, + /// Vector dimension. + dim: usize, + /// Distance metric. + metric: Metric, + /// Max degree. + max_degree: usize, +} + +impl std::fmt::Debug for JniProvider { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("JniProvider") + .field("dim", &self.dim) + .field("metric", &self.metric) + .field("max_degree", &self.max_degree) + .field("graph_size", &self.graph.len()) + .finish() + } +} + +// SAFETY: JavaVM is Send+Sync, GlobalRef is Send+Sync. +unsafe impl Send for JniProvider {} +unsafe impl Sync for JniProvider {} + +impl JniProvider { + /// Build a new search-only provider from pre-constructed graph data. + /// + /// * `graph_data`: (internal_id, external_id, neighbors) tuples. + /// * `start_id`: internal ID of the graph start point. + /// * `start_vec`: vector for the start point (kept in memory). + /// * `jvm`: Java VM reference for JNI callbacks. + /// * `reader_ref`: global ref to the Java vector reader. + /// * `dim`: vector dimension. + /// * `metric_type`:metric enum value (0=L2, 1=IP, 2=Cosine). + /// * `max_degree`: maximum adjacency list size. + pub fn new( + graph_data: Vec<(u32, i64, Vec)>, + start_id: u32, + start_vec: Vec, + jvm: JavaVM, + reader_ref: GlobalRef, + dim: usize, + metric_type: i32, + max_degree: usize, + ) -> Self { + let graph = DashMap::new(); + let mut int_to_ext = HashMap::new(); + let mut ext_to_int = HashMap::new(); + + for (int_id, ext_id, neighbors) in &graph_data { + let adj = AdjacencyList::from_iter_untrusted(neighbors.iter().copied()); + graph.insert( + *int_id, + GraphTerm { + neighbors: adj, + ext_id: *ext_id, + }, + ); + int_to_ext.insert(*int_id, *ext_id); + ext_to_int.insert(*ext_id, *int_id); + } + + // Ensure start point is in the graph too. + if !graph.contains_key(&start_id) { + graph.insert( + start_id, + GraphTerm { + neighbors: AdjacencyList::new(), + ext_id: start_id as i64, + }, + ); + int_to_ext.insert(start_id, start_id as i64); + ext_to_int.insert(start_id as i64, start_id); + } + + let mut start_points = HashMap::new(); + start_points.insert(start_id, start_vec); + + Self { + graph, + int_to_ext, + ext_to_int, + start_points, + jvm, + reader_ref, + graph_reader_ref: None, + dim, + metric: map_metric(metric_type), + max_degree, + } + } + + /// Build a search-only provider with on-demand graph reading (no pre-loaded graph data). + /// + /// The graph `DashMap` starts empty and acts as a lazy cache — entries are populated + /// on first access via JNI callback to `graphReader.readNeighbors(int)`. + pub fn new_with_readers( + int_to_ext: HashMap, + ext_to_int: HashMap, + start_id: u32, + start_vec: Vec, + jvm: JavaVM, + reader_ref: GlobalRef, + graph_reader_ref: GlobalRef, + dim: usize, + metric_type: i32, + max_degree: usize, + ) -> Self { + let graph = DashMap::new(); + + // The start point needs an entry so `to_internal_id` works. + // Its neighbors will be fetched on demand. + let mut start_points = HashMap::new(); + start_points.insert(start_id, start_vec); + + Self { + graph, + int_to_ext, + ext_to_int, + start_points, + jvm, + reader_ref, + graph_reader_ref: Some(graph_reader_ref), + dim, + metric: map_metric(metric_type), + max_degree, + } + } + + pub fn dim(&self) -> usize { + self.dim + } + + pub fn metric(&self) -> Metric { + self.metric + } + + /// Fetch neighbor list from Java via JNI callback to `graphReader.readNeighbors(int)`. + /// Returns None if graphReader is not set. + fn fetch_neighbors_jni(&self, int_id: u32) -> Result>, JniProviderError> { + let graph_ref = match &self.graph_reader_ref { + Some(r) => r, + None => return Ok(None), + }; + + let mut env = self + .jvm + .attach_current_thread() + .map_err(|e| JniProviderError::JniCallFailed(format!("attach failed: {}", e)))?; + + let result = env.call_method( + graph_ref, + "readNeighbors", + "(I)[I", + &[jni::objects::JValue::Int(int_id as i32)], + ); + + let ret_val = match result { + Ok(v) => v, + Err(e) => { + let _ = env.exception_clear(); + return Err(JniProviderError::JniCallFailed(format!( + "readNeighbors({}) failed: {}", + int_id, e + ))); + } + }; + + let obj = match ret_val.l() { + Ok(o) => o, + Err(_) => return Ok(Some(Vec::new())), + }; + + if obj.is_null() { + return Ok(Some(Vec::new())); + } + + let int_array = jni::objects::JIntArray::from(obj); + let len = env + .get_array_length(&int_array) + .map_err(|e| JniProviderError::JniCallFailed(format!("get_array_length: {}", e)))? + as usize; + + let mut buf = vec![0i32; len]; + env.get_int_array_region(&int_array, 0, &mut buf) + .map_err(|e| JniProviderError::JniCallFailed(format!("get_int_array_region: {}", e)))?; + + Ok(Some(buf.into_iter().map(|v| v as u32).collect())) + } + + /// Fetch a vector from Java via JNI. Returns None if readVector returns null. + fn fetch_vector_jni(&self, ext_id: i64) -> Result>, JniProviderError> { + let mut env = self + .jvm + .attach_current_thread() + .map_err(|e| JniProviderError::JniCallFailed(format!("attach failed: {}", e)))?; + + let result = env.call_method( + &self.reader_ref, + "readVector", + "(J)[F", + &[jni::objects::JValue::Long(ext_id as jlong)], + ); + + let ret_val = match result { + Ok(v) => v, + Err(e) => { + // Clear any pending Java exception so the JNI env stays usable. + let _ = env.exception_clear(); + return Err(JniProviderError::JniCallFailed(format!( + "readVector({}) failed: {}", + ext_id, e + ))); + } + }; + + let obj = match ret_val.l() { + Ok(o) => o, + Err(_) => return Ok(None), + }; + + if obj.is_null() { + return Ok(None); + } + + // Convert JFloatArray → Vec. + let float_array = jni::objects::JFloatArray::from(obj); + let len = env + .get_array_length(&float_array) + .map_err(|e| JniProviderError::JniCallFailed(format!("get_array_length: {}", e)))? + as usize; + + let mut buf = vec![0f32; len]; + env.get_float_array_region(&float_array, 0, &mut buf) + .map_err(|e| { + JniProviderError::JniCallFailed(format!("get_float_array_region: {}", e)) + })?; + + Ok(Some(buf)) + } +} + +// ======================== Context ======================== + +/// Lightweight execution context (mirrors test_provider::Context). +#[derive(Debug, Clone, Default)] +pub struct JniContext; + +impl std::fmt::Display for JniContext { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "jni context") + } +} + +impl provider::ExecutionContext for JniContext {} + +// ======================== DataProvider ======================== + +impl provider::DataProvider for JniProvider { + type Context = JniContext; + type InternalId = u32; + type ExternalId = u32; + type Error = JniProviderError; + + fn to_internal_id( + &self, + _context: &JniContext, + gid: &u32, + ) -> Result { + // Check ID mapping (works for both in-memory and on-demand modes). + if self.int_to_ext.contains_key(gid) || self.start_points.contains_key(gid) { + Ok(*gid) + } else { + Err(JniProviderError::InvalidId(*gid)) + } + } + + fn to_external_id( + &self, + _context: &JniContext, + id: u32, + ) -> Result { + if self.int_to_ext.contains_key(&id) || self.start_points.contains_key(&id) { + Ok(id) + } else { + Err(JniProviderError::InvalidId(id)) + } + } +} + +// ======================== SetElement (stub — search only) ======================== + +impl provider::SetElement<[f32]> for JniProvider { + type SetError = ANNError; + type Guard = provider::NoopGuard; + + async fn set_element( + &self, + _context: &JniContext, + _id: &u32, + _element: &[f32], + ) -> Result { + Err(ANNError::opaque(JniProviderError::JniCallFailed( + "set_element not supported on search-only JniProvider".to_string(), + ))) + } +} + +// ======================== NeighborAccessor ======================== + +#[derive(Debug, Clone, Copy)] +pub struct JniNeighborAccessor<'a> { + provider: &'a JniProvider, +} + +impl provider::HasId for JniNeighborAccessor<'_> { + type Id = u32; +} + +impl provider::NeighborAccessor for JniNeighborAccessor<'_> { + async fn get_neighbors( + self, + id: Self::Id, + neighbors: &mut AdjacencyList, + ) -> ANNResult { + // 1. Try in-memory graph (populated upfront or cached from previous JNI calls). + if let Some(term) = self.provider.graph.get(&id) { + neighbors.overwrite_trusted(&term.neighbors); + return Ok(self); + } + + // 2. On-demand: fetch from Java graph reader via JNI callback. + if self.provider.graph_reader_ref.is_some() { + let fetched = self.provider.fetch_neighbors_jni(id)?; + if let Some(neighbor_ids) = fetched { + let adj = AdjacencyList::from_iter_untrusted(neighbor_ids.iter().copied()); + neighbors.overwrite_trusted(&adj); + // Cache in the DashMap for subsequent accesses. + let ext_id = self.provider.int_to_ext.get(&id).copied().unwrap_or(id as i64); + self.provider.graph.insert(id, GraphTerm { neighbors: adj, ext_id }); + return Ok(self); + } + } + + Err(ANNError::opaque(JniProviderError::InvalidId(id))) + } +} + +impl provider::NeighborAccessorMut for JniNeighborAccessor<'_> { + async fn set_neighbors(self, id: Self::Id, neighbors: &[Self::Id]) -> ANNResult { + match self.provider.graph.get_mut(&id) { + Some(mut term) => { + term.neighbors.clear(); + term.neighbors.extend_from_slice(neighbors); + Ok(self) + } + None => Err(ANNError::opaque(JniProviderError::InvalidId(id))), + } + } + + async fn append_vector(self, id: Self::Id, neighbors: &[Self::Id]) -> ANNResult { + match self.provider.graph.get_mut(&id) { + Some(mut term) => { + term.neighbors.extend_from_slice(neighbors); + Ok(self) + } + None => Err(ANNError::opaque(JniProviderError::InvalidId(id))), + } + } +} + +// ======================== DefaultAccessor ======================== + +impl provider::DefaultAccessor for JniProvider { + type Accessor<'a> = JniNeighborAccessor<'a>; + + fn default_accessor(&self) -> Self::Accessor<'_> { + JniNeighborAccessor { provider: self } + } +} + +// ======================== Accessor ======================== + +/// Accessor that fetches vectors via JNI callback. +/// +/// Keeps a local buffer and an LRU cache to reduce JNI round-trips. +pub struct JniAccessor<'a> { + provider: &'a JniProvider, + buffer: Box<[f32]>, + cache: VectorCache, +} + +impl<'a> JniAccessor<'a> { + pub fn new(provider: &'a JniProvider, cache_size: usize) -> Self { + let buffer = vec![0.0f32; provider.dim()].into_boxed_slice(); + Self { + provider, + buffer, + cache: VectorCache::new(cache_size), + } + } +} + +impl provider::HasId for JniAccessor<'_> { + type Id = u32; +} + +impl provider::Accessor for JniAccessor<'_> { + type Extended = Box<[f32]>; + type Element<'a> = &'a [f32] where Self: 'a; + type ElementRef<'a> = &'a [f32]; + type GetError = JniProviderError; + + async fn get_element( + &mut self, + id: u32, + ) -> Result, Self::GetError> { + // 1. Start-point vectors are always in memory. + if let Some(vec) = self.provider.start_points.get(&id) { + self.buffer.copy_from_slice(vec); + return Ok(&*self.buffer); + } + + // 2. Check LRU cache. + if let Some(cached) = self.cache.get(id) { + self.buffer.copy_from_slice(cached); + return Ok(&*self.buffer); + } + + // 3. JNI callback to Java: FileIOVectorReader.readVector(extId). + let ext_id = self + .provider + .int_to_ext + .get(&id) + .copied() + .unwrap_or(id as i64); + + let fetched = self.provider.fetch_vector_jni(ext_id)?; + + match fetched { + Some(vec) if vec.len() == self.provider.dim() => { + self.buffer.copy_from_slice(&vec); + self.cache.put(id, vec.into_boxed_slice()); + Ok(&*self.buffer) + } + Some(vec) => Err(JniProviderError::JniCallFailed(format!( + "readVector({}) returned {} floats, expected {}", + ext_id, + vec.len(), + self.provider.dim() + ))), + None => Err(JniProviderError::InvalidId(id)), + } + } +} + +// ======================== DelegateNeighbor ======================== + +impl<'this> provider::DelegateNeighbor<'this> for JniAccessor<'_> { + type Delegate = JniNeighborAccessor<'this>; + + fn delegate_neighbor(&'this mut self) -> Self::Delegate { + JniNeighborAccessor { + provider: self.provider, + } + } +} + +// ======================== BuildQueryComputer ======================== + +impl provider::BuildQueryComputer<[f32]> for JniAccessor<'_> { + type QueryComputerError = diskann::error::Infallible; + type QueryComputer = ::QueryDistance; + + fn build_query_computer( + &self, + from: &[f32], + ) -> Result { + Ok(::query_distance( + from, + self.provider.metric(), + )) + } +} + +// ======================== BuildDistanceComputer ======================== + +impl provider::BuildDistanceComputer for JniAccessor<'_> { + type DistanceComputerError = diskann::error::Infallible; + type DistanceComputer = ::Distance; + + fn build_distance_computer( + &self, + ) -> Result { + Ok(::distance( + self.provider.metric(), + Some(self.provider.dim()), + )) + } +} + +// ======================== SearchExt ======================== + +impl glue::SearchExt for JniAccessor<'_> { + fn starting_points( + &self, + ) -> impl std::future::Future>> + Send { + futures_util::future::ok(self.provider.start_points.keys().copied().collect()) + } +} + +// ======================== Blanket traits ======================== + +impl glue::ExpandBeam<[f32]> for JniAccessor<'_> {} +impl glue::FillSet for JniAccessor<'_> {} + +// ======================== Strategy ======================== + +/// Search-only strategy for the JNI provider. +#[derive(Debug, Default, Clone, Copy)] +pub struct JniStrategy; + +impl JniStrategy { + pub fn new() -> Self { + Self + } +} + +impl glue::SearchStrategy for JniStrategy { + type QueryComputer = ::QueryDistance; + type PostProcessor = glue::CopyIds; + type SearchAccessorError = diskann::error::Infallible; + type SearchAccessor<'a> = JniAccessor<'a>; + + fn search_accessor<'a>( + &'a self, + provider: &'a JniProvider, + _context: &'a JniContext, + ) -> Result, diskann::error::Infallible> { + // Cache up to 1024 recently fetched vectors to reduce JNI round-trips. + Ok(JniAccessor::new(provider, 1024)) + } + + fn post_processor(&self) -> Self::PostProcessor { + Default::default() + } +} + +// For insert (graph construction) — delegates to prune/search accessors. +// We implement InsertStrategy and PruneStrategy as stubs since the JniProvider +// is search-only. DiskANNIndex::new() requires the Provider to be Sized but +// does NOT call insert methods unless we invoke index.insert(). +impl glue::PruneStrategy for JniStrategy { + type DistanceComputer = ::Distance; + type PruneAccessor<'a> = JniAccessor<'a>; + type PruneAccessorError = diskann::error::Infallible; + + fn prune_accessor<'a>( + &'a self, + provider: &'a JniProvider, + _context: &'a JniContext, + ) -> Result, Self::PruneAccessorError> { + Ok(JniAccessor::new(provider, 1024)) + } +} + +impl glue::InsertStrategy for JniStrategy { + type PruneStrategy = Self; + + fn prune_strategy(&self) -> Self::PruneStrategy { + *self + } + + fn insert_search_accessor<'a>( + &'a self, + provider: &'a JniProvider, + _context: &'a JniContext, + ) -> Result, Self::SearchAccessorError> { + Ok(JniAccessor::new(provider, 1024)) + } +} + +impl<'a> glue::AsElement<&'a [f32]> for JniAccessor<'a> { + type Error = diskann::error::Infallible; + fn as_element( + &mut self, + vector: &'a [f32], + _id: Self::Id, + ) -> impl std::future::Future, Self::Error>> + Send { + std::future::ready(Ok(vector)) + } +} diff --git a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs index 5c3df469a148..21d12def0732 100644 --- a/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs +++ b/paimon-diskann/paimon-diskann-jni/src/main/native/src/lib.rs @@ -30,7 +30,7 @@ //! undefined behaviour and likely crash the JVM. On panic the function throws a //! `java.lang.RuntimeException` with the panic message and returns a safe default. -use jni::objects::{JByteArray, JByteBuffer, JClass, JPrimitiveArray, ReleaseMode}; +use jni::objects::{JByteArray, JByteBuffer, JClass, JObject, JPrimitiveArray, ReleaseMode}; use jni::sys::{jfloat, jint, jlong}; use jni::JNIEnv; @@ -43,6 +43,9 @@ use diskann::graph::{self, DiskANNIndex}; use diskann::neighbor::{BackInserter, Neighbor}; use diskann_vector::distance::Metric; +mod jni_provider; +use jni_provider::{JniContext, JniProvider, JniStrategy}; + // ======================== Constants ======================== const METRIC_L2: i32 = 0; @@ -51,8 +54,8 @@ const METRIC_COSINE: i32 = 2; /// Serialization magic number ("PDAN"). const MAGIC: i32 = 0x5044414E; -/// Serialization format version (2 = real DiskANN). -const SERIALIZE_VERSION: i32 = 2; +/// Serialization format version (3 = graph + vectors). +const SERIALIZE_VERSION: i32 = 3; /// The u32 ID reserved for the DiskANN graph start/entry point. const START_POINT_ID: u32 = 0; @@ -83,7 +86,7 @@ where // ======================== Metric Mapping ======================== -fn map_metric(metric_type: i32) -> Metric { +pub(crate) fn map_metric(metric_type: i32) -> Metric { match metric_type { METRIC_INNER_PRODUCT => Metric::InnerProduct, METRIC_COSINE => Metric::Cosine, @@ -233,14 +236,6 @@ fn read_i64(buf: &[u8], offset: &mut usize) -> Option { Some(i64::from_ne_bytes(b)) } -fn read_f32(buf: &[u8], offset: &mut usize) -> Option { - if *offset + 4 > buf.len() { return None; } - let mut b = [0u8; 4]; - b.copy_from_slice(&buf[*offset..*offset + 4]); - *offset += 4; - Some(f32::from_ne_bytes(b)) -} - fn write_i32(buf: &mut [u8], offset: &mut usize, v: i32) -> bool { if *offset + 4 > buf.len() { return false; } buf[*offset..*offset + 4].copy_from_slice(&v.to_ne_bytes()); @@ -262,10 +257,6 @@ fn write_f32(buf: &mut [u8], offset: &mut usize, v: f32) -> bool { true } -fn serialization_size(dimension: i32, count: usize) -> usize { - 8 * 4 + count * (8 + (dimension as usize) * 4) -} - // ======================== JNI Functions ======================== #[no_mangle] @@ -607,6 +598,67 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearch< } } +// ============================================================================ +// Serialization format (graph + vectors) +// ============================================================================ +// +// Layout: +// Header : 9 × i32 (magic, version=3, dimension, metric, index_type, +// max_degree, build_list_size, count, start_id) +// Graph : for each node (ordered by ascending internal id): +// ext_id : i64 +// int_id : i32 +// neighbor_cnt : i32 +// neighbors : neighbor_cnt × i32 +// Vectors : for each node (same order): +// vector : dim × f32 +// +// During search-only mode the Rust side loads the graph into a `JniProvider` +// and invokes the Java `FileIOVectorReader.readVector(long)` callback +// for every vector access that is NOT a start point. +// ============================================================================ + +// ---- Searcher registry (handles backed by JniProvider) ---- + +struct SearcherState { + index: Arc>, + context: JniContext, + runtime: tokio::runtime::Runtime, + dimension: i32, + /// Mapping from internal DiskANN u32 IDs to external Java long IDs. + int_to_ext: HashMap, +} + +struct SearcherRegistry { + next_handle: i64, + searchers: HashMap>>, +} + +impl SearcherRegistry { + fn new() -> Self { + Self { next_handle: 100_000, searchers: HashMap::new() } + } + fn insert(&mut self, state: SearcherState) -> i64 { + let h = self.next_handle; + self.next_handle += 1; + self.searchers.insert(h, Arc::new(Mutex::new(state))); + h + } +} + +fn searcher_registry() -> &'static Mutex { + static REG: OnceLock> = OnceLock::new(); + REG.get_or_init(|| Mutex::new(SearcherRegistry::new())) +} + +fn get_searcher(handle: i64) -> Option>> { + searcher_registry().lock().ok()?.searchers.get(&handle).cloned() +} + +// ======================== indexSerialize ======================== + +/// Serialize the index with its graph adjacency lists. +/// Returns the number of bytes written. #[no_mangle] pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSerialize<'local>( mut env: JNIEnv<'local>, @@ -616,184 +668,662 @@ pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSeriali ) -> jlong { let arc = match get_index(handle) { Some(a) => a, - None => { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid index handle"); - return 0; - } + None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid handle"); return 0; } }; let state = match arc.lock() { Ok(s) => s, - Err(_) => { - let _ = env.throw_new("java/lang/IllegalStateException", "Index lock poisoned"); - return 0; - } + Err(_) => { let _ = env.throw_new("java/lang/IllegalStateException", "Lock poisoned"); return 0; } }; - let count = state.raw_data.len(); - let required = serialization_size(state.dimension, count); + // Collect graph data from the underlying DiskANN test_provider. + let provider = state.index.provider(); - let buf = match get_direct_buffer_slice(&mut env, &buffer, required) { - Some(slice) => slice, - None => { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Buffer too small"); - return 0; - } - }; + let dim = state.dimension as usize; + + // Build ordered list of (int_id, ext_id, neighbors) using the async + // NeighborAccessor API run synchronously on our tokio runtime. + // Include the start point (int_id=0) so the search can navigate from it. + let mut graph_section_size: usize = 0; + let mut graph_entries: Vec<(u32, i64, Vec)> = Vec::new(); - let mut offset = 0usize; - if !write_i32(buf, &mut offset, MAGIC) - || !write_i32(buf, &mut offset, SERIALIZE_VERSION) - || !write_i32(buf, &mut offset, state.dimension) - || !write_i32(buf, &mut offset, state.metric_type) - || !write_i32(buf, &mut offset, state.index_type) - || !write_i32(buf, &mut offset, state.max_degree as i32) - || !write_i32(buf, &mut offset, state.build_list_size as i32) - || !write_i32(buf, &mut offset, count as i32) + // Start point first (ext_id = -1 as sentinel — never a real user ID). { - let _ = env.throw_new("java/lang/IllegalStateException", "Serialize header failed"); - return 0; + use diskann::graph::AdjacencyList; + use diskann::provider::{DefaultAccessor, NeighborAccessor as NeighborAccessorTrait}; + let accessor = provider.default_accessor(); + let mut adj = AdjacencyList::::new(); + let mut neighbors = Vec::new(); + if state.runtime.block_on(accessor.get_neighbors(START_POINT_ID, &mut adj)).is_ok() { + neighbors = adj.iter().copied().collect(); + } + graph_section_size += 8 + 4 + 4 + neighbors.len() * 4; + graph_entries.push((START_POINT_ID, -1i64, neighbors)); } - for (id, vector) in &state.raw_data { - if !write_i64(buf, &mut offset, *id) { - let _ = env.throw_new("java/lang/IllegalStateException", "Serialize failed"); - return 0; + // User-added vectors. + for (ext_id, _vec) in &state.raw_data { + let int_id = match state.ext_to_int.get(ext_id) { + Some(i) => *i, + None => continue, + }; + let mut neighbors = Vec::new(); + { + use diskann::graph::AdjacencyList; + use diskann::provider::{DefaultAccessor, NeighborAccessor as NeighborAccessorTrait}; + + let accessor = provider.default_accessor(); + let mut adj = AdjacencyList::::new(); + if state.runtime.block_on(accessor.get_neighbors(int_id, &mut adj)).is_ok() { + neighbors = adj.iter().copied().collect(); + } } - for &v in vector { - if !write_f32(buf, &mut offset, v) { - let _ = env.throw_new("java/lang/IllegalStateException", "Serialize failed"); - return 0; + + graph_section_size += 8 + 4 + 4 + neighbors.len() * 4; // ext_id + int_id + cnt + neighbors + graph_entries.push((int_id, *ext_id, neighbors)); + } + + let count = graph_entries.len(); // includes start point + let vector_section_size = count * dim * 4; + let header_size = 9 * 4; // 9 i32s + let total_size = header_size + graph_section_size + vector_section_size; + + let buf = match get_direct_buffer_slice(&mut env, &buffer, total_size) { + Some(s) => s, + None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Buffer too small"); return 0; } + }; + + let mut off = 0usize; + // Header + write_i32(buf, &mut off, MAGIC); + write_i32(buf, &mut off, SERIALIZE_VERSION); + write_i32(buf, &mut off, state.dimension); + write_i32(buf, &mut off, state.metric_type); + write_i32(buf, &mut off, state.index_type); + write_i32(buf, &mut off, state.max_degree as i32); + write_i32(buf, &mut off, state.build_list_size as i32); + write_i32(buf, &mut off, count as i32); + write_i32(buf, &mut off, START_POINT_ID as i32); + + // Graph section + for (int_id, ext_id, neighbors) in &graph_entries { + write_i64(buf, &mut off, *ext_id); + write_i32(buf, &mut off, *int_id as i32); + write_i32(buf, &mut off, neighbors.len() as i32); + for &n in neighbors { + write_i32(buf, &mut off, n as i32); + } + } + + // Vector section (same order as graph_entries) + for (int_id, ext_id, _) in &graph_entries { + if *int_id == START_POINT_ID { + // Write the start point's dummy vector. + for _ in 0..dim { + write_f32(buf, &mut off, 1.0); + } + } else if let Some((_, vec)) = state.raw_data.iter().find(|(id, _)| id == ext_id) { + for &v in vec { + write_f32(buf, &mut off, v); + } + } else { + // Zero-fill for missing vectors. + for _ in 0..dim { + write_f32(buf, &mut off, 0.0); } } } - required as jlong + total_size as jlong } +/// Return the serialized size. #[no_mangle] pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSerializeSize<'local>( mut env: JNIEnv<'local>, _class: JClass<'local>, handle: jlong, ) -> jlong { - jni_catch_unwind(&mut env, 0, AssertUnwindSafe(|| { - get_index(handle) - .and_then(|arc| { - arc.lock() - .ok() - .map(|s| serialization_size(s.dimension, s.raw_data.len()) as jlong) - }) - .unwrap_or(0) - })) + let arc = match get_index(handle) { + Some(a) => a, + None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid handle"); return 0; } + }; + let state = match arc.lock() { + Ok(s) => s, + Err(_) => { let _ = env.throw_new("java/lang/IllegalStateException", "Lock poisoned"); return 0; } + }; + + let dim = state.dimension as usize; + + // Calculate size by iterating over all graph nodes (start point + raw_data). + let provider = state.index.provider(); + let mut graph_section_size: usize = 0; + + // Start point. + { + use diskann::graph::AdjacencyList; + use diskann::provider::{DefaultAccessor, NeighborAccessor as NeighborAccessorTrait}; + let accessor = provider.default_accessor(); + let mut adj = AdjacencyList::::new(); + let nc = if state.runtime.block_on(accessor.get_neighbors(START_POINT_ID, &mut adj)).is_ok() { + adj.len() + } else { 0 }; + graph_section_size += 8 + 4 + 4 + nc * 4; + } + + // User-added vectors. + for (ext_id, _) in &state.raw_data { + let int_id = match state.ext_to_int.get(ext_id) { + Some(i) => *i, + None => continue, + }; + let neighbor_count = { + use diskann::graph::AdjacencyList; + use diskann::provider::{DefaultAccessor, NeighborAccessor as NeighborAccessorTrait}; + let accessor = provider.default_accessor(); + let mut adj = AdjacencyList::::new(); + if state.runtime.block_on(accessor.get_neighbors(int_id, &mut adj)).is_ok() { + adj.len() + } else { + 0 + } + }; + graph_section_size += 8 + 4 + 4 + neighbor_count * 4; + } + + let count = state.raw_data.len() + 1; // +1 for start point + let header_size = 9 * 4; + let vector_section_size = count * dim * 4; + (header_size + graph_section_size + vector_section_size) as jlong } +// ======================== indexCreateSearcher ======================== + +/// Create a search-only handle from serialized data + Java callback reader. +/// +/// `data`: byte[] containing the serialized index. +/// `vectorReader`: Java object with a `readVector(long)` method (e.g. `FileIOVectorReader`). +/// +/// Returns a searcher handle (≥100000) that can be used with `indexSearchWithReader`. #[no_mangle] -pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexDeserialize<'local>( +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateSearcher<'local>( mut env: JNIEnv<'local>, _class: JClass<'local>, data: JByteArray<'local>, - _length: jlong, + vector_reader: JObject<'local>, ) -> jlong { let bytes = match env.convert_byte_array(&data) { - Ok(data) => data, - Err(_) => { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid data"); - return 0; - } + Ok(d) => d, + Err(_) => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid data"); return 0; } }; - let mut offset = 0usize; + let mut off = 0usize; - let magic = match read_i32(&bytes, &mut offset) { - Some(v) => v, - None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Data too short"); return 0; } - }; + // Parse header. + let magic = read_i32(&bytes, &mut off).unwrap_or(0); if magic != MAGIC { - let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid magic number"); + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid magic"); return 0; } - - let version = match read_i32(&bytes, &mut offset) { - Some(v) => v, - None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Data too short"); return 0; } - }; - - let dimension = read_i32(&bytes, &mut offset).unwrap_or(0); - let metric_type = read_i32(&bytes, &mut offset).unwrap_or(METRIC_L2); - let index_type = read_i32(&bytes, &mut offset).unwrap_or(0); - - let (max_degree, build_list_size, count) = if version >= 2 { - let md = read_i32(&bytes, &mut offset).unwrap_or(64); - let bls = read_i32(&bytes, &mut offset).unwrap_or(100); - let cnt = read_i32(&bytes, &mut offset).unwrap_or(0) as usize; - (md, bls, cnt) - } else if version == 1 { - let cnt = read_i32(&bytes, &mut offset).unwrap_or(0) as usize; - (64, 100, cnt) - } else { - let _ = env.throw_new("java/lang/IllegalArgumentException", format!("Unsupported version: {}", version)); + let version = read_i32(&bytes, &mut off).unwrap_or(0); + if version != SERIALIZE_VERSION { + let _ = env.throw_new("java/lang/IllegalArgumentException", + format!("Expected version {}, got {}", SERIALIZE_VERSION, version)); return 0; - }; + } + let dimension = read_i32(&bytes, &mut off).unwrap_or(0); + let metric_type = read_i32(&bytes, &mut off).unwrap_or(0); + let _index_type = read_i32(&bytes, &mut off).unwrap_or(0); + let max_degree = read_i32(&bytes, &mut off).unwrap_or(64) as usize; + let build_ls = read_i32(&bytes, &mut off).unwrap_or(100) as usize; + let count = read_i32(&bytes, &mut off).unwrap_or(0) as usize; + let start_id = read_i32(&bytes, &mut off).unwrap_or(0) as u32; + let dim = dimension as usize; + + // Parse graph section. + let mut graph_data: Vec<(u32, i64, Vec)> = Vec::with_capacity(count); + let mut int_to_ext: HashMap = HashMap::with_capacity(count); - let dim = dimension as usize; - let mut entries: Vec<(i64, Vec)> = Vec::with_capacity(count); for _ in 0..count { - let id = match read_i64(&bytes, &mut offset) { + let ext_id = match read_i64(&bytes, &mut off) { Some(v) => v, - None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated data"); return 0; } + None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated graph"); return 0; } }; - let mut vector = Vec::with_capacity(dim); - for _ in 0..dim { - let v = match read_f32(&bytes, &mut offset) { - Some(val) => val, - None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated data"); return 0; } + let int_id = match read_i32(&bytes, &mut off) { + Some(v) => v as u32, + None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated graph"); return 0; } + }; + let ncnt = match read_i32(&bytes, &mut off) { + Some(v) => v as usize, + None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated graph"); return 0; } + }; + let mut neighbors = Vec::with_capacity(ncnt); + for _ in 0..ncnt { + let n = match read_i32(&bytes, &mut off) { + Some(v) => v as u32, + None => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Truncated graph"); return 0; } }; - vector.push(v); + neighbors.push(n); } - entries.push((id, vector)); + int_to_ext.insert(int_id, ext_id); + graph_data.push((int_id, ext_id, neighbors)); } - let mut state = - match create_index_state(dimension, metric_type, index_type, max_degree, build_list_size) { - Ok(s) => s, - Err(msg) => { - let _ = env.throw_new("java/lang/RuntimeException", format!("Failed to create index during deserialization: {}", msg)); - return 0; + // Create JNI global ref for the vector reader callback (needed before + // we fetch the start-point vector below). + let global_reader = match env.new_global_ref(&vector_reader) { + Ok(g) => g, + Err(e) => { + let _ = env.throw_new("java/lang/RuntimeException", + format!("Failed to create global ref: {}", e)); + return 0; + } + }; + + let jvm = match env.get_java_vm() { + Ok(vm) => vm, + Err(e) => { + let _ = env.throw_new("java/lang/RuntimeException", + format!("Failed to get JavaVM: {}", e)); + return 0; + } + }; + + // Fetch start-point vector via JNI callback. + // The byte[] passed to this function only contains header + graph + // (no vector section), so vectors are read on-demand from the reader. + // The start point has ext_id = -1 (sentinel); its vector is the dummy + // [1.0, …] written during serialization. We try the reader first, + // falling back to a non-zero dummy to avoid division-by-zero with + // cosine metric. + let start_ext_id = graph_data.iter() + .find(|(iid, _, _)| *iid == start_id) + .map(|(_, eid, _)| *eid) + .unwrap_or(-1); + let start_vec = { + // Try reading from the Java reader callback. + let fetched = (|| -> Option> { + let mut e = jvm.attach_current_thread().ok()?; + let result = e.call_method( + &global_reader, "readVector", "(J)[F", + &[jni::objects::JValue::Long(start_ext_id)], + ).ok()?; + let arr = result.l().ok()?; + if arr.is_null() { return None; } + let jarr: jni::objects::JFloatArray = arr.into(); + let len = e.get_array_length(&jarr).ok()? as usize; + if len == 0 { return None; } + let mut buf = vec![0.0f32; len]; + e.get_float_array_region(&jarr, 0, &mut buf).ok()?; + Some(buf) + })(); + fetched.unwrap_or_else(|| vec![1.0f32; dim]) + }; + + // Build the JniProvider (graph in memory, vectors via JNI callback). + let provider = JniProvider::new( + graph_data, + start_id, + start_vec, + jvm, + global_reader, + dim, + metric_type, + max_degree, + ); + + // Build DiskANNIndex config. + let md = std::cmp::max(max_degree, 4); + let bls = std::cmp::max(build_ls, md); + let metric = map_metric(metric_type); + + let index_config = match graph::config::Builder::new( + md, + graph::config::MaxDegree::same(), + bls, + metric.into(), + ).build() { + Ok(c) => c, + Err(e) => { + let _ = env.throw_new("java/lang/RuntimeException", + format!("Failed to create index config: {:?}", e)); + return 0; + } + }; + + let runtime = match tokio::runtime::Builder::new_current_thread() + .enable_all() + .build() + { + Ok(r) => r, + Err(e) => { + let _ = env.throw_new("java/lang/RuntimeException", + format!("Failed to create tokio runtime: {}", e)); + return 0; + } + }; + + // DiskANNIndex takes ownership of the provider. After construction, + // the provider is accessible via index.provider() for read access. + // We store int_to_ext separately for mapping search results. + let index = Arc::new(DiskANNIndex::new(index_config, provider, None)); + + let searcher = SearcherState { + index, + context: JniContext, + runtime, + dimension, + int_to_ext, + }; + + match searcher_registry().lock() { + Ok(mut guard) => guard.insert(searcher), + Err(_) => { + let _ = env.throw_new("java/lang/IllegalStateException", "Searcher registry error"); + 0 + } + } +} + +// ======================== indexCreateSearcherFromReaders ======================== + +/// Create a search-only handle from two on-demand Java readers: one for graph +/// structure and one for vectors. +/// +/// `graphReader`: Java object with `readNeighbors(int)`, `getDimension()`, +/// `getCount()`, `getStartId()`, `getMaxDegree()`, +/// `getBuildListSize()`, `getMetricValue()`, +/// `getAllInternalIds()`, `getAllExternalIds()`. +/// `vectorReader`: Java object with `readVector(long)`. +/// +/// Returns a searcher handle (≥100000) for use with `indexSearchWithReader`. +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexCreateSearcherFromReaders<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + graph_reader: JObject<'local>, + vector_reader: JObject<'local>, +) -> jlong { + // Helper to call int-returning methods on graphReader. + macro_rules! call_int { + ($name:expr) => { + match env.call_method(&graph_reader, $name, "()I", &[]) { + Ok(v) => match v.i() { Ok(i) => i, Err(_) => { let _ = env.throw_new("java/lang/RuntimeException", concat!("Bad return from ", stringify!($name))); return 0; } }, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("Failed to call {}: {}", $name, e)); return 0; } } + } + } + + let dimension = call_int!("getDimension"); + let metric_type = call_int!("getMetricValue"); + let _index_type = call_int!("getIndexTypeValue"); + let max_degree = call_int!("getMaxDegree") as usize; + let build_ls = call_int!("getBuildListSize") as usize; + let count = call_int!("getCount") as usize; + let start_id = call_int!("getStartId") as u32; + let dim = dimension as usize; + + // Read int_to_ext mapping via getAllInternalIds() / getAllExternalIds(). + let int_ids: Vec = { + let result = match env.call_method(&graph_reader, "getAllInternalIds", "()[I", &[]) { + Ok(v) => v, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("getAllInternalIds failed: {}", e)); return 0; } }; + let obj = match result.l() { + Ok(o) => o, + Err(_) => { let _ = env.throw_new("java/lang/RuntimeException", "Bad return from getAllInternalIds"); return 0; } + }; + let arr = jni::objects::JIntArray::from(obj); + let len = env.get_array_length(&arr).unwrap_or(0) as usize; + let mut buf = vec![0i32; len]; + let _ = env.get_int_array_region(&arr, 0, &mut buf); + buf + }; - let strat = test_provider::Strategy::new(); - for (ext_id, vector) in &entries { - let int_id = state.next_id; - state.next_id += 1; - state.ext_to_int.insert(*ext_id, int_id); - state.int_to_ext.insert(int_id, *ext_id); - state.raw_data.push((*ext_id, vector.clone())); + let ext_ids: Vec = { + let result = match env.call_method(&graph_reader, "getAllExternalIds", "()[J", &[]) { + Ok(v) => v, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("getAllExternalIds failed: {}", e)); return 0; } + }; + let obj = match result.l() { + Ok(o) => o, + Err(_) => { let _ = env.throw_new("java/lang/RuntimeException", "Bad return from getAllExternalIds"); return 0; } + }; + let arr = jni::objects::JLongArray::from(obj); + let len = env.get_array_length(&arr).unwrap_or(0) as usize; + let mut buf = vec![0i64; len]; + let _ = env.get_long_array_region(&arr, 0, &mut buf); + buf + }; + + // Build ID mappings. + let mut int_to_ext: HashMap = HashMap::with_capacity(count); + let mut ext_to_int: HashMap = HashMap::with_capacity(count); + for i in 0..std::cmp::min(int_ids.len(), ext_ids.len()) { + let iid = int_ids[i] as u32; + let eid = ext_ids[i]; + int_to_ext.insert(iid, eid); + ext_to_int.insert(eid, iid); + } + + // Create global refs for both readers. + let global_graph_reader = match env.new_global_ref(&graph_reader) { + Ok(g) => g, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("graph ref: {}", e)); return 0; } + }; + let global_vector_reader = match env.new_global_ref(&vector_reader) { + Ok(g) => g, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("vector ref: {}", e)); return 0; } + }; + + let jvm = match env.get_java_vm() { + Ok(vm) => vm, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("get JVM: {}", e)); return 0; } + }; + + // Fetch start-point vector via vectorReader JNI callback. + let start_ext_id = int_to_ext.get(&start_id).copied().unwrap_or(-1); + let start_vec = { + let fetched = (|| -> Option> { + let mut e = jvm.attach_current_thread().ok()?; + let result = e.call_method( + &global_vector_reader, "readVector", "(J)[F", + &[jni::objects::JValue::Long(start_ext_id)], + ).ok()?; + let arr = result.l().ok()?; + if arr.is_null() { return None; } + let jarr: jni::objects::JFloatArray = arr.into(); + let len = e.get_array_length(&jarr).ok()? as usize; + if len == 0 { return None; } + let mut buf = vec![0.0f32; len]; + e.get_float_array_region(&jarr, 0, &mut buf).ok()?; + Some(buf) + })(); + fetched.unwrap_or_else(|| vec![1.0f32; dim]) + }; + + // Build the JniProvider with on-demand graph reading. + let provider = JniProvider::new_with_readers( + int_to_ext.clone(), + ext_to_int, + start_id, + start_vec, + jvm, + global_vector_reader, + global_graph_reader, + dim, + metric_type, + max_degree, + ); + + // Build DiskANNIndex config. + let md = std::cmp::max(max_degree, 4); + let bls = std::cmp::max(build_ls, md); + let metric = map_metric(metric_type); + + let index_config = match graph::config::Builder::new( + md, + graph::config::MaxDegree::same(), + bls, + metric.into(), + ).build() { + Ok(c) => c, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("config: {:?}", e)); return 0; } + }; + + let runtime = match tokio::runtime::Builder::new_current_thread() + .enable_all() + .build() + { + Ok(r) => r, + Err(e) => { let _ = env.throw_new("java/lang/RuntimeException", format!("runtime: {}", e)); return 0; } + }; + + let index = Arc::new(DiskANNIndex::new(index_config, provider, None)); + + let searcher = SearcherState { + index, + context: JniContext, + runtime, + dimension, + int_to_ext, + }; + + match searcher_registry().lock() { + Ok(mut guard) => guard.insert(searcher), + Err(_) => { let _ = env.throw_new("java/lang/IllegalStateException", "Registry error"); 0 } + } +} + +// ======================== indexSearchWithReader ======================== + +/// Search on a searcher handle created by `indexCreateSearcher`. +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexSearchWithReader<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + handle: jlong, + n: jlong, + query_vectors: JPrimitiveArray<'local, jfloat>, + k: jint, + search_list_size: jint, + distances: JPrimitiveArray<'local, jfloat>, + labels: JPrimitiveArray<'local, jlong>, +) { + let num = n as usize; + let top_k = k as usize; + + let arc = match get_searcher(handle) { + Some(a) => a, + None => { + let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid searcher handle"); + return; + } + }; + + // Copy query vectors. + let query: Vec = { + let elems = match unsafe { env.get_array_elements(&query_vectors, ReleaseMode::NoCopyBack) } { + Ok(a) => a, + Err(_) => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Invalid queries"); return; } + }; + elems.iter().copied().collect() + }; + + let state = match arc.lock() { + Ok(s) => s, + Err(_) => { let _ = env.throw_new("java/lang/IllegalStateException", "Lock poisoned"); return; } + }; + + let dimension = state.dimension as usize; + let total = num * top_k; + let mut result_dist = vec![f32::MAX; total]; + let mut result_lbl = vec![-1i64; total]; + + let strat = JniStrategy::new(); + + for qi in 0..num { + let qvec = &query[qi * dimension..(qi + 1) * dimension]; + let search_k = top_k + 1; + let l = std::cmp::max(search_list_size as usize, search_k); + + let params = match graph::SearchParams::new(search_k, l, None) { + Ok(p) => p, + Err(e) => { + let _ = env.throw_new("java/lang/IllegalArgumentException", + format!("Invalid search params: {}", e)); + return; + } + }; + + let mut neighbors = vec![Neighbor::::default(); search_k]; let idx_clone = Arc::clone(&state.index); - let ctx = &state.context; - let result = panic::catch_unwind(AssertUnwindSafe(|| { - state.runtime.block_on(idx_clone.insert(strat, ctx, &int_id, vector.as_slice())) + let search_result = panic::catch_unwind(AssertUnwindSafe(|| { + state.runtime.block_on(idx_clone.search( + &strat, + &state.context, + qvec, + ¶ms, + &mut BackInserter::new(&mut neighbors), + )) })); - match result { - Ok(Ok(())) => {} + let stats = match search_result { + Ok(Ok(s)) => s, Ok(Err(e)) => { - let _ = env.throw_new("java/lang/RuntimeException", format!("Deserialization insert failed for id {}: {}", ext_id, e)); - return 0; + let _ = env.throw_new("java/lang/RuntimeException", + format!("Search failed: {}", e)); + return; } Err(_) => { - let _ = env.throw_new("java/lang/RuntimeException", format!("Deserialization insert panicked for id {}", ext_id)); - return 0; + let _ = env.throw_new("java/lang/RuntimeException", "Search panicked"); + return; } + }; + + let rc = stats.result_count as usize; + let mut cnt = 0; + for ri in 0..rc { + if cnt >= top_k { break; } + let nb = &neighbors[ri]; + if nb.id == START_POINT_ID { continue; } + let idx = qi * top_k + cnt; + result_lbl[idx] = state.int_to_ext.get(&nb.id).copied().unwrap_or(nb.id as i64); + result_dist[idx] = nb.distance; + cnt += 1; } } - match registry().lock() { - Ok(mut guard) => guard.insert(state), - Err(_) => { - let _ = env.throw_new("java/lang/IllegalStateException", "DiskANN registry error"); - 0 - } + drop(state); + + // Write back distances. + { + let mut de = match unsafe { env.get_array_elements(&distances, ReleaseMode::CopyBack) } { + Ok(a) => a, + Err(_) => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Bad distances"); return; } + }; + for i in 0..std::cmp::min(de.len(), result_dist.len()) { de[i] = result_dist[i]; } + } + // Write back labels. + { + let mut le = match unsafe { env.get_array_elements(&labels, ReleaseMode::CopyBack) } { + Ok(a) => a, + Err(_) => { let _ = env.throw_new("java/lang/IllegalArgumentException", "Bad labels"); return; } + }; + for i in 0..std::cmp::min(le.len(), result_lbl.len()) { le[i] = result_lbl[i]; } } } + +/// Destroy a searcher handle. +#[no_mangle] +pub extern "system" fn Java_org_apache_paimon_diskann_DiskAnnNative_indexDestroySearcher<'local>( + mut env: JNIEnv<'local>, + _class: JClass<'local>, + handle: jlong, +) { + jni_catch_unwind(&mut env, (), AssertUnwindSafe(|| { + if let Ok(mut guard) = searcher_registry().lock() { + guard.searchers.remove(&handle); + } + })); +} diff --git a/paimon-diskann/paimon-diskann-jni/src/main/resources/darwin/aarch64/libpaimon_diskann_jni.dylib b/paimon-diskann/paimon-diskann-jni/src/main/resources/darwin/aarch64/libpaimon_diskann_jni.dylib deleted file mode 100755 index 556f8cdbc79a062fa831a8774421652ad223d6aa..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1024496 zcmeF43w%`7wg2~-31sp_NFV{MB&3!ADk{%WkeNg!yrfF%vzo++Oh8193P_a+Y0D&` zg(E=(yf;B^H50T_g*H>_HOQ^S>#c%RZR@QDsC5E9(D2CcF#qp5a}F8e1X1zPe{VjY z$vJzUefD$hwbovH?caIj@cW-6Dy1p@y7D`N-@^{2exjC2l z{O3xD8ty`@{K(4qD=4^o+T6R%0GE#W16@u-gZ zx$X?-^JjZFiG(BbbC~(jcH=LcUx8=ZZFfw&i$?en4n2z~<_aEW>Mmxg^ ztgw|q$GpjJ_U)zZ1JW_S!hUhatE1-yK&Oq^l%s4vp~wROyM#Pv|(v zub|*kcfPwIcghu0!_;iiuh&f7AuM^yPwFajqo82=?Dk&kJU>Zm{E=T|y=0DEWj>P2 zr^n&M`ki6cXWs+imn0mSUqn{Mt+aD4l?}Hu;%TEfPhDnQhp^SoSo3i~!EHqarSoRp zJo~nSJMO-#)BKiNgGhdn^^$zT9URWEpx~CGyNk@sxpb`G(&xkV+ZP{BBjL#W+TRS{ z!B0GQO)I*k;NGIUgbQ@c&-=%4ext(Vp(GrcUrL7zkLPDrvtxb@FNgE1HQ$qPWPaT{ z%kSn=gGqJF@7hqWC$C55*W*+9nHib!9rJT+59hZv;=RcHB8o9m2;I$^HT!le zdB^;=nfdvw1YA#+pU_<%zk-5EuSRv2SzOri>hjYyT&XqjlFk;kj^9VEv&NV;kS{ zUr1aAzi?kuf~}8nZ;XQWa?Bu}9p_ox%P5WFCwYaZxXkESHRPVsyQFvISAqz+_q+=u zyCuqdlTZV+6W*ITuk`L|v(C$jeDQnaF6GZnN>u5zeV`eZ{*u4%m|@q#Zfbo$BkFVg zb0}9RPVIZOpBA^bk*;lybw4flF}+=f%Lq5x3tm?)rI)N#sVc@5(hfYZW1;e1HTNNv z>PXFTI+E0t{d%f9AGN9c{Zszae{8MnZFP-3&@{%-_-|su>&i{$w+FRXOYY&GrVpaZJ zrBeGP5tmF{cXelzTzSXM)$`~?r^IDaU(Zh`hwI-(-5I2Aw`mS;xZ|5aa-}}t? z2Z;aZexv=PB>p_&?X>U0_V^2lhc;6CWvV;97vjBy*(K0Zg6OZl33(jet87pAv5_uD-z4?C&lTFr_{a70 zUR5}wq}8IGX3}`>GT*(u{oUOXf3q2XQ+xd1B+veK{W0=fkQ48;Xhi74a|L~ItZjou zx2|Jt`4&A3{T4nNumAQ@RsHw2wzRl>?^SPc-4AW{VLap-tuHrIUXwkg{wWo1^8;p^ z-~PF(hNT(rS=Yk;G5TWCNn4FRd!W7EPj%8xzqIFhn~v&I4G%H$lXf-PQ|qVpwpH8G zZPlNmd)MVogQkVvp&yy6Leovq)(QRb=(q^_9)BWz_cH0GPdnZGRO)!xtotX_-J~0@ z)bVDU{wQts)J%+a_!SIDoa4UyyFnB)p&GGB$qHQ62Y z!Via$x6q6=7LIem^-|}z_vzr}o$xe=*#?=nCLhQ!+TYF(-~D&^?Je$b-Qv`pyXa%b z8D<%&?eF#ItPJ?P(N9u_ImWTlPf>V-Ch>jD^bx$_NHy_{QBZh#1aGj~+)}Mlt$Uf9 zZRNJ@tp#2E2d{~Rr#p}hamWUne!y<4KWNwLW&X9$e;@u%ReQUt4NbAiAG9kyn5FsK zQdH>RHFkef7Z%*!k-Eq!rHy_40Pl=cORpM5pC43e-|NpO>2>`XKX`bP%nAAiv6|mx z?^(ao-mCtGrCN1Ef6ZCf!|n9sOn&@DuP9tk>+p8 zQlVe(wrvmY)^x{}v3hWb3hn2jDp;y}Cbze@r#Pd8(5BB5v^YNs&`5jr|F)9=}O&<$is4Gq# zKp*A#t-me*L2Ki}#?gL>FWWqN z^TLIrgYd@93!fidW~&$-oUcMnDgb_Vq=Mh~LClessZolo4j zNk-fsyNBZn&mwMz*|)^`GS#rPHkBG2;0_6|ZXTe)x8wGz2`c~a5aNcoLK*JH(H8G^ z(-%^Ha9NlaG%ac8Gp);*8|7yFg7)~2iTCE%^iBV!sxM|7ocuzJlXsWWCw|hl;wUtB zz}}}mv^&XJSDfEI-}c(J|E?MNo}ljBoFj8Bt=7ORrUq-(?CVj}f@^ZhExVlklL(5bx6ugEqZ_q*m7duLowxoRv|TUY4MtwJntQazV2R`V`u$ct6FCr5?$@J`SkUmurrY3ro3U7?A_ zyNP>myI?WzChpw@%}sKhibH>y*EqvJCc9m-n!u`e*)t-DJib%T>9*1-Iepa zE=pf~fzsc%C)dBNCG1@YFJ1Bz*Qj3-Kf5hG=5WeZ(6634aPhu$D# zq8(|=FN4=*ss!Bw&Hjmg=#zQlz!LHwTIQ$^PIZSo32Ng_v_-}w?-%3z(tmp|c8A^@ z>kduO;D6KU%bC#7d}wC@^i<9@4H{a+IF3tnw$0W;mE^all<*vWxrxqT0q+!t``?v? zo^hG0LE?h*>7ONB`$6Z+NHftMXd9a3tc9-27RKv2c2)gMdH$#cJ=C3RXsgJXAZ7Tt z`l#cAWa8nYO}yU{>k0`^pQS@*Wv9d{%-o0n0S=??hRE2D!0;tYP}U>*p~?ChcPh0g{__Ycb5v@CyA-R}IkFZOpe zIO(_F+JDd*oE_FTn<&@PYxX>-4E`*24%$-&IWqg!r?`6?{Ve)P(>0_u^^>=)w9G%x zS*nNdJ{2G4Z5lM@`J`}1aH1^`Tz8qj591tsiTXe1g7zJIvoG+iYG#f$&2fbW!Yk_A zhWYCrAMO|aF!QGHTx}XkJDEeFi8+A-Q*r`_bH_WI;3e-=(O;`g-d5Aj+cr_3XuNQv z#Fu}X7d}ONC`|=~AI^D1RrjU8o>`buzv&b72g-TLEXVO$+lHoya@O$PTGGUIS9&w^ zG&q&9fp#^`707_zTt97vo&OIvJ#wv>c0o!H{^SGyZBr$O|hGFfZNSkaF{ zCqek}JgztJi{!&QceLf#QtqSO?(yd!(}K{q>VjSq7uIWL8QU}6u@4Es6 zyQuDQ@2fzNv1`7;ZSa)L!)^Hi@~X?VZFgiUy~%E`H|CiuUhmtgRX?+Lj^Ci4h_Q8l z)RzBenSU}D9@j#LOYMPRU*^G6TIk4H_}VkX&1612ljQvEmh^@a=vCxr#u}xs*`aOU zUuMjW))sV0k)b6ij6Hmy1YNQ)fpLTm`YCO%yQj+6Xp@KeVcY+}4$~E2-9tt#wo9*G&c}>${ot4Osh|J!LxTGYeX+~xH8Vle%Iw!T6oUA@iSG8 zfoSAKq`!``^SF+M-^iRb_`mR+!UX-$-Ksi|dN`7LSEXnw;OL^B&4yRzB|y)Q)1T|Y z{aJLVE&sdpjnQY+N9LXMZF^o0{r89HSNk=iga?$t1A+y-UmSj4`&(OnwE5T-S{n!* zG&3J%?1V?6Tx|$3mKSx4tID)PbFr>xg>Q~$%$-?EA8YTk*WE^bW_rw0shd)+8C6$& z&D<9s%?Z!{@@tIwpWW29F@}0Okc%ZBxB}UlBf;|lv?8=q<_g3&y8`32xP-W7;*gm$ zWNuJa=25MB4eu?apK@yB^@mby)pIB(kGlHMt35V!9psZnI}6b*(0N%aUXC7o8T#<0 z=*79{$2tC^v)%sYtUug#&QauMH~2%EE713BHSo;WSZj#hNghRviRdR;N845evt%sb z8_u2RFy1X|=tyfRP=tf-|Q}s1=?^QbWok)F+XKIlCDC?M<+JSmm7kB{AF14No z?dE$u(7(M`9iaS$JRi+-BR_A7et)X1dK6)q%WIg+UeV1P6ZGE^kNq;$2-kMk8wiW8 zXoWWu9%O|{|14qbiQ#;t&5}=3oL)~nYvS;Ggr6aNX2*2Oq1O^GI;rs3`Q~UFiIvzwXlD=4L-ivN) z*c$1ByhLbwwmalVL|#P5mZ`|O8%Xcrhs`v#oB6Kfa~1bD^JA=!=Y0qsEO9^V5H~zZ zSo9&QPV%1gGdB6uOZZJPpIJ1S*)2?yOOCXyxYvAU)xpYRd?)eO5pUy{-O2OFdPtkB z{IH1|_mTWd%9Oe`GBz@Ag76LF3h#)<3+(it@f=>Z{bMST zFHvOhU&%L`4E~eEmw#FYzf8Qy-xaIOwwY^@Y|1gpjvE&-kU76RUA3N-RP9=A`%cQc+=ia8`}+2E*hFpn^|ZgP{JK%1Zq{S0{UWwiNH+MY`v8#!Y6&y zyUwPtbEehP7oDcHA8tz%JCDp^xk~y^LTxK#K13?}gRoA3hd< z8KeRWo>oq&hv))pkS(64T~73`;d2JL0&`z?`wtC3H(87Qf#1$$O22-Asy@j2t{3b7 zL-d8pRr-%ss_Ne8IIVBx_@ASUI%uHH#+sk~s9gsi*inbwRmJtLl6kp~c4XVrs_IDR zLEqeJ>YMNZL*MjJA5Xk1Bx}$L`mwp6D^SrxH8i)l8!Ec0fR8@UmOg$}`uIfq;^FC+ z9k(%}Z*R-2g+dNA(d# z&vH_~tNy{b@9E|W>|rglr^VI4I<{dfebt1H&pJ1-ryud9?uI=rT7#^gWv*%T#V*P$ z=lvh^d*CCjIzXGAHJZ+F$w(fss-IK18 zd(jz0&k%Vj@5q=(_UC4kcm15Qgg-pdoqZM3bEiPyvx1p}QcLVm+K;F=<4M(8kV*1FK1FY>mA2SEKtKs)v{Yg=@+G8KtjgUgRO;x5i*=&QkvT!LCpZ?HkGDgiG z(eQd#r9Z^8M`G1S8Q3)&E_XGoWM8<+-rKO-?_!=5bk(XAHXNTVr}b^-lKYdkeD6Zm zdg#)b8NKzR536c<=RN4Cu$b}*`>1C>Ep6{X)>In#-b0)B!|Oz!GUh`g_8R(McJ8hj-mdY=m2Uj_YN$vpTzynBkDd13Hw z>A!zJs8zq-aSj)D(H~--iR?D!ZEdXnAYtqN5$m3GE4deXFz7TRMbDs3W%Oa+Gs1IQ zcycm7(X(Z}AUdhcb&X%1*&oK0`gXDAFZVX?End(J9gIB|?$@34N5}VM?jNKcO{AMa zds=DFqmn95WBO3a3Oifo=yn+1^P*<_)^>k9~NJVqFc56 zhfK!;9RZhFjFe|Bfd;{taue%|7%7dqQtaa<1MIuZzC;gzP=g z_H)x!e%k`OzYpd0TO=MEff1k5Zufgu;;};+@$gB@SK>DkKfsKK*I72s zUlH$R-rq5a>y{XO`%g)?#7tM!p6+oYU5tJQdq2MU=JN&Z&ub+AQZs&bd;Cg?FE--~ z+T)i=epzQ+38rzUXXHD<56h zPyvHaW^~o-%CRp&dxKk$-S(?T)umiLO7h&8b@kle=3d>fq5SIBR;4yJLD%w5zoVw?qy5I*g%6JH z7T(JrD{GrX&Cp4kk(Jq6STDVezVpFbS5?^ptCrXT6&Jfh3*oQAFTJ_(y6oF-$4{x5 zGDF3=&ftPv=bjyt=jOQ`gZ3<&+>pmU#eU{jQ~ArIYwZt>jypqbsI|X7x@n|$%mMn^ zpPbMd=&zh{$L`qSzeKKuJN&Pzxpfb!h7IWMwF}EeuZdsam-0mCEk1>^N|M!virFf( zvfS?Ur6e~<-W4@=r-QtGQ8(L{bY|$WMSN~ydCk! zuH7MX;xK!12kde6N1)q^`P6SW^;_00T)#H6emlC&tuCkBDE&O=6xwd}a|P=rtDh?t zQC4n@(Z_qJ_xRr7ejXpE=H+1zuU?`;-ih8Z%PF&#{oYsLZDK!JR*o)4_(da}Z6y)o3%*!;cydSqF*Qv_v`dDO3(C%I3>#K#9YNj<9NNvR<+F z3g=jO`xA^oMT#0OdP1(aicq=yEo^*O81Ifmc>8Lw=Da6aAwyr5pOm6hqeR zA?;$?h)rU+JfFz(vB;z%$`Rh!#&aWYQ&y}KnPWcpwOrv0lK=8I-Jw;Ras%>y7x+pc zI=*SsU!P;z^dF%QFlweYArId$_9=SS|Bd@Ua6juFt-7|WGGu67j&cgmt3a3Z8Z>Z* zA;a9vxmS=aZ}JnFCbk&kxwotSB60VVw-GMvq5q!nJS)uemkHlvg)`#xX9?dK5q^R2 zEW(nn_|_Qth%KM^>1Mo}y%Hndm83sOe3A8jBes0PH%5e?CVZV0CZAst{sCcZkRchz zBMFQpWohVu;X2#(he5iApzQNCl&mms&G2YEc)GLS^bNt;{{U?NR zx(N4&E7o{txD_USCC@Lg!i9GIr-aY7!j!R)@Ib;+w#@Nxf28QMiO;gq(e9aq(KW;6 z5WbsmUn{J7=*5K7BEq*3PO-x5Wq1g8BP=#78S?_>j&(l{o+9_MKkzr=uZ1qHXM2&) za{r;3{z~)uCo^91dMDyufoeS~b zXd>R>LPyT(GVj;4!Ftxud{^@6Z{EvZkUSTkS&6sat25sV5~qsd^;!Gz@8O>POCx={ zdG$o-1oB+$lA;GhlmBj>^p(RKR=7h| ztKsoU=~W^RD_B=mv979QT_t*H&F%zeG#xYpy|d3xTn(a|u0TE>M*nTX#vpR!?{mX^ zTzF){(Rn+X2f6%(Nm}yYBDH}SochKJLrFhDBoE^W~H=DYngVx}K;wwhS#D_=p zY|*hZLv8uKO*w&wpUVjxLcf)Lq9AsuAi7TWg2D$^@J^c6r~Y9x58n;&@Htv&Rf{c9 zvm?PN`hOYgvj2?39=iiyNSROYF~-!e<;z&X*HS~fC;L~jkD=%*2W!&o?rLKn)~CmP zpYT&#JrjLJWW#KCXc_N0?j$?}-2*u^gZ&=MFRGCHM)V}1IT`z%)LZu2ZxniiXRM|i z&j6QykalLJDw%Oq)nz0n()aISoB4qCSQULglfIvjG_h)e7FU^2vS-x9gw_#@(QhYb zs_lN_5=#ad`t3RBw{i62_+xRa9q79y=)^M;&_lGimrCGq%|kdJQFc~Ca?x17ryI17 z9_;C+LYcHVn>GyO$9}nCZ%1b2qnnOk{jomN8y8+bVaIy=!|?h^^w~_(EJvUBLa)iJ z*P3RkP&st_leEcBp}B?jME%G1BtxH^RjR5#WPW?I7WkJHk3}C)8>}^EI(;N_spELB zX>QB^G4vthvz2=JxLP{%R_<%yBhl7z(iTZCylxltFSdeA;$Cjns_!Mg1kHPu^z8&K z!RXV-exC`w-iB{lh;;Zr9Fy^gXFMh_4ijV?j^*x1D48*;mgk3s=aaA09UWtR4rfbt zlJCDK*s7m}UpVMj8F$$`6uXt&i;YOGVgr(E9`BW>Q2Tc#NN+= zEa(ckAIi;j%KD*jzRH*WFT&TcW=Ep4jCoQf>;EkY`obNF`V)&(^~2mRXI*>%`=!XH z{n$oQpyw6LiIwn}RqU5vpVeEJy8Z8c&IUFObm4uy%kT%ZtE)57M;iBecQ$N$f}mHVDHfggG5vqxGr(gZKoR@4ZWZNk1JC8x?)D zjP;V#$G2mW(=$$usHRRn){~+GiZ6GKP3=99g`IXV^y@-L<5@*kf?hc^H&BJ{v>2Lu z_)U1@+gfPxZusN7@Qc0h$T#f)Yt3D=DR$lm(D}i=F@6X0z%c~B|5#V3@@@F*Zt{3D zH;^6Y2>ah%Y!56RYY$YcC7;L1Zyk9(1@Az&t67`utSKhnwMlv-cG4H)(1)JW@Tu^Q zsh*h|=*^sy{r<)9xXhBh^^_9IY4U{B7ff|cgQz}wc2xG|Cx+2D!&(gNxn_w zv(6nl7$GOFpVPUVc#pB(jjVV<>H^<&%*3zC-fNKXm4WvcJ~&jvKbtx|O#PNoKcDE( zuV4>cYY)^cOAhnj;$;0MM?3%3@PYF%2lDLUIUu&OgL^OabI>log|@ZxphS}grPcrE z2kks)^B~&yF71kAE^_s~=?>W#AI(Ai@UwSl+Diq>u{ZBl^d`pr@A#=TU4y-W@n2Eg z4V?$Rlf)SSk(2O=KD6Iq%E*#al#wpx$cNYiWS$;io^4~i&*;(k*Y!!?uR6mPU)lYW z#MO6}+{t)%pUL65{B4YPvMqVf5!S^AZn)GhH16&oJ3nxQ$G(=ak0zJP@b?zk+4}Zm zf9u&{Srz<*y}4;>qx4&deL9g>GAA9=RKDoA)myNY{UI!`9(oD63eR}XC$g%yv5wlv zy{kAin6K_ni5%6 zfjy#^$f+BV6O50@DIYeSBg|p(8*8Gkg~vCigzYCqG3*TnHNBj@l;T?KZP?0o#UPXS zgl%QQD{Jv%`3rM2Xg}knROV}+vr|_Kt(>{Wj--^TUZxKl z`@(uQ{a0h}yEnt3@^>Ql9~uWwq#mqEXS&3H4BnQq6aQN3yY-VcgPs=M8=aOC(X&pC zPox@r;%!^EnL=Zcd}0^#FmrMLfN@rU=aer3L?V!K`gpAW$s zRxy6};X6}c@4Yu~=*m?#=-kU*vDlQvCp`9QRlRdIa>AiLdKB3uzIQSQJm_F68S}%? znve7qH!Hoezou6`k3C`L2)}>GNPoROy|tV<`;$3SoIBTYmP_op=p%n;&ThO5n*p*y z_(mjMY-LOxTC(oYurDJAeaMaqWXGVhFY{L+OR8$%@oSMG@cX7kmp|!U)aS`&Y}Xa= zMc;gQRtvt$lq2%^sn)g?VmB7~TZ@feXsYP~_O?o|9#v18@mIR$W@Dck-`&-ajeSb& z$&oe{u}6tr$~PZdb+X!c4|KMJ`jtMKs2An9oY^s)J%#t(%sRi2{q}kI;Cm0Z4U>CE zqtYw3uui1z>+EXZOx>ybhm7zeySEMO-bJ5~sp<6tM*6WuEgh(xq0f-H!TAzD^_n2} z*pv3OCHtEhhf2~+C`#6w*Twrq-`hi9HV<~-R~YBt$-LS2AFBG-jB!2o@pah7>#&Xc zY0EjJeV_UE1olelgNy_0m&e+idtPWm9btJ*kB<{J4*J;uZTE#B|m)@H~AYkqI#`EOX~MULmzY{nDaA?Q#e zjB#Dy4po*T*UfPiIbO-QTH|Qw{*2=o#?jKFY8gL+H)M%_X)g7iY}R|SF^2o0)p+Xc zpw7~c1B~Bce4V8Jb)s)krlrG9N9Xdf7BJ#xw#OF{?=4Tz2jP#_qeC0ScAQN+EImi^ z^&D&4_yPUzE6xq9n4TMG5!u?YfBw*x-$0tO=C)ytjHjwi)SqTQRy$Mmkh!US&^G)s zo>x*oSx+WtDSIy+=sH};*$C00jQ)c5#RmVwxP+=l)*!EcL0`?Y2UdKPq}P0uL<0(BfsMLMxNivoY;=P-Q(=NLO0$=k(tY&8DysLk05glUyP6;2Mb)G2HC&1 zr>rYx|Kx1?p$VQWdU#*CKhM~EUhmj=iD4&HG3rUjNA6a~KCLxKx}cr&QP3@V_0k+= zH)Ox8DO|iSZEWf03O)WFd(g~#V?XC4^Y9|*tm$!=>;)~&EYx)2N0q!QbnuzuzRu&GdsmYqX!lI1vzg}8~Z%!xKz%3kT8BU9XdOFue< zJj!6bxs3hCM&!59QH9)}WztdEn06nAo%k>`c}H(0-3fHmsGUGZY4qo3(a!PtZ_+_z ze`d0$n@v4@_@D|OaiMD)yb&HAWGxqbBgb#-NAyF-WN!Q3YTI}S`>H&P#zU@_z11i@ zE&_Nwn8B596 zvTZo9Pk0#1NZUotJ6iQGC{KLQVtD?yceU#OAuRLx!}qo7o0vyUGtsT`)knMNv;L&} zeo@%2^4Kxc=7C<@na$&zM?c%q$6Ne= z#8xA^>iOJ@Ek`rs#8xf7Jdy87+8)GN{yh>Fo3+$a?hW2XUQ!ncGxv-uH1S|Fa?SGb zvFyj9t7jv3^Mpp=gRF^-JsIgc@riBXo!g;-1N8l4^iz8osrAxU3Cp|i>~J3Dx=HLX zLX+7&Pt;|G{X|v&hvbW#EXzvN?}}+(uNRs2(w&UOt|gO=vsJ$7tOFv>R(Y{KOTHRs zt7cDurxA`kTjfJ9sX#9gn{HV@_HLndk&A|W=1i5WW8{q0f3besu=ZZR_)q*7<$aHI zXY>0mzbt+t$7NqUfpzPt?u!qM+^?Okv0s}Iv0oc&?2C`Du+PietOcsEBYQV_$5?i$ zB6u(3T;Ch{m~QP&#)tPNMYh%9@3EJ?+LiRRtWC<~N|^nL4LW%fsTj<9Tyg!KC@t_BZ z?per~T0Hi7%J!t0@r=0@|2yIfwLbm=OWTqX`zKn&>@#$Q(Hm4 zL+t_av#NO`(YX|QWuCJqDSNZxuaaIrr7QEFJx!rmnP-oi^y{VnrhOPb=OFyDN$i;) zw5@o8=gqtm&9`VF?TqGI^dsWMx9Bq3FaB(M@xz$MoXT6^m^ywz`qT;i98>ErNgrDB z9c1j4YOeU8#(zf*ozY!2+}(xs3~ls{6YQ27@S(dNU%Kn?sk;{6x@+(&yqdMt4`$(W zXZRL6T%^tFGVcrrdvjX)-W=>&ao95|(D8hO(W~jtT@mYZrL}VNZTUm&qcTqfBzuQn#r?b zh>QW_Gcr7ul}}Cai?6}rV!QscEjC^J)oK`Lp#d4sD#r3>jOAkHO2tgERsRKk5kp_BqQ0{CCUY}X9`5H_>@_RRdD+A|Pv+*!qz}uWF~@1M zj(q+g`9O0P|4%gOQ}}C`F4OC4piAEnb1gWOHkv+pZ&6+}AHF{m-^5z%1>O^1yX861 zCu4H(M?$AD`s0!xb_ru`xuAU>|F`5?y^Q5b-m~aW^n}^ShO?L}hs#yK zk4`20Q_h>p`r=ylXO5sh+yJi%l4qo!R4sNL>%3_Zx}=;pt;0@Zoj0u~-`~A%o;S6J z&zsi53uT@dvg@N{=S7=y14Ca!|Djwh8&A;+Jle9;{in0(M-(wMBngfDu~6Fg==H?sE1i|A+LesEYv5Syf9Ip^7s3o;HF zE8Fc77egaP+9Bbz!hichFVbh?ry_MJr7p7XphNfB(jO^FPH$g$75rr30QQ^)yF-u2 zzMt4&;9)(%-CCYTe?A9qdxrHt_QPGW|FG5-dXD=%`fB-Fo;?*2eoBRwKSSK6a9rJm zzE!?WDkQ$rzTbr7O30&(JbbHZ5B7ZFe|h)@mywU}@%A_maW&+>ysekNrmd&n7t5Y3 zd~sPDG$i(#Bb<@^9`p`|mmQj|iPb_vD-QPb;+UhH=Y45GfAx_^>R^XQz5*|Jliwlg zx0iFzQTV|B^1KcjwrrDmN$3Ylasm}u z*s$o=D&~9@JXrSqE3bi9zG(~8l$!h3@aM%X?KZ5<^r=tA9~mZkVUYXlC|BgVhL4Qp zS2YS*Cw@cX!z_M8KFX?~Eb%XJ2%n}bFJ+0YA%2&Q&$Vqde7&o30+PlUf71uq@yI6| zckgE%+lkFA((Y;b;zi?otE8XMXX|8aV&O3+pRAVnZf1Oh-tu#a?`y{QBfW|9=975z z9wQ%U-_mmy5-6U*U5xI=kIdk+&$eOPYkz(R&x5Qro?*;N z!L5t5*V?d2$oe#iGnq{vMc8Z6k652>leKl#ZhVqhS2KUti@kOow%QW%8kpH#H|(?6 zYQ;V){6cKCVyB(K^OMH|BAhNYyDYtq;2?2o{Joj^C#B2 zamwGd>I)@4tIP3tIny+r_zQ?1cBm~)c+B^qp*()$_+8HLGJX<2!Mqy%^itc#LNhFD z+ziez8M^+!PvfXOk30|@6_NRSu!ZM=nR3qAuF&A#@OAJ|1T&={*M+2$vxV57mO69K z)$gNi*V_|xS?AP&zmhUfze#DU+^%=)K#-KJjp5gS0%dp zK<1^$Hv9n%To;M=t%Xmm<*dYSNVAD|SKHxF=Kd~w0a^@efOZ~faG1Vn*3|nr=cxX>)f<0Nv|-=s(KFjvV}JNfqJ~e z58p{SA6A2$_Wg!;H^I|hp-wx(I%ip9TUZZ)FG%^8tX?C&M95-;XHcHSGwuO*Lv&jD z$SG^omC$Q5{COE`)_aoUs*2e2E9@#5620piCNc;6D}8~jr-30MwsuK>3+-zehn@B* z_q^t;$%gzFoF_~EcP#T-$~5>ZWj5PW2JNx;95g;Nu6~$1)fn#=g+Ee$rpX^CndwE} zTnUzptO*r*nS-;la{eR}oxm|50{f3Ka7+yIJvF7BuU$kw4#q|5qlOZ1=0A*hLw>N= zX~~br;EkepKRnVMnu|>B#~z=<+T%mV4e#-#*C$Vj*yH2fmA)LlxRQ0KuP=~t@%F5+fnh!_jgnZHV<;ql4WAA7P;GxMmHTClHV}T#jf>J?l*9+3H{L5$Mv_T+cqqvEfTkfdRfnxn{n?% z#2I$w)gr&z>(xlT{!E&-at#c_PoL$_KA-(Z?330x(2@8}jBvOD2V>dOL$(F8RCt|` zqVy~7V6PRP@S0%ibYs0`o~LN$JVjH)Iw2F?L)Hnwh;>43jIFx$uiPr4;WFXmeOaEyK{<-g!c)!(7NgpMD@PWvB}`@5}LwO~n-N7yDn|5~_ojg)QS zy>`SY9{aNsaf+|-;#2koaEe`we>9xpi-<3Lz8#lStLy~R({;IlEmMoGBm70`d^+%2 z`$)YGhxt!o1YT>Z)az!OK4Kovk&D^TT3iRb)&$as-IX;X^6BJwt?$FTqG6~WM&5V^ zndN`4oiFYuzVPODyw*(>r;gY9j^uffndgZ1Jl~c)Bk)=mLzfprSEmE7bt`$x`rg7T zjfU6y+=5QV)?wC_d1}eP4MzLh>0&_ZD93H-)LtW%j2r6$VtCI;O(a(8kw+seoN$ zTc+y89&;E|;rFZoz9~$FfjilYMSfVA3i;SrA~6*V-d@;CzmPc-&%8Int~2%X2wV4~ ztb5W8=U&z$VxJSaBJWsy-@>Ta3?GyIYA0pcd47%7Ii^A_`57|eRptCQ?!PHa1%DJw z1rOus*x5VmbI=K*hhCW&ty;42&?btV+dRKeKAah|GeMYGYVrWJVW?j8B^hI+8@m>dAr23<`Lb* z8)299NPHB#0)&8my5RlGePn%HS?d{ zt`l7<@x^9*L3{iql3y0*E68sib4k{;Jvk%uc`+3XKI!cn-Vec6`T8!?kBWT{$@ANe zsSxJ#eGNZx@$Y?%`kxF_VIlG)CjwL9u}`%BZ!r~q18@5#Fck{$2m2;46@J9J;#7S{ zzAUCf@+p+{c`y}5kl#NGrb7QyX#45FRJae`7Ky2Fw-Np{ro#PT6@OEh3ge)4!BnU< z&eNY7rUE+tdPf;vHq zbWHn~!c=exro!SxoqkxKlQGD^NghWZ2&O_6dc1|H;NxB}71F0g$5cS?U9U2}cid)q zAF}1A{KSSSm$R+#j| z2@9rz@f`oM3kVCQf)S>Sa|uUcDj4yufqEa}v#fNqyBA^3rG(!j+>dZyD_l50PbMtq zjKld3(9;N~SYh%@A^gvdsZc~4WbZ+ILp~3t!c_A;*~bt}g&Z(;C6DZ{08?QUWr@xH z33NHZRH)}FmEQZg6sqi$uum6Xs@XcTsN`P&Q(*>JOX43SKIVnoN5fRO z5xwNwjH%!jOogp+dNFiTtj;!UWxF`P^$h%8Y-Qpf^jVk+b9pyCxBqFyRJgDcT8f5w z;AQ>vY3!A67p6k}_0cgECcx)U1ykYa%Od=(CWyb)>A_Seyg53i0)ETiYD@)ubUVdV z_%3HhyiqU}s!qgI*f&tK9miBCA-{n|XP@*eDZaW(Cv48@6jNb7-Yfm8Pkn zyAMMYOa-GqGty3=qvzi=_X1CX$MJPyDrmGP8V@;gPdlc<7WO8x=_B!{jyzXuVJd8Z z{yM`{SiV|zwN1DDh#GiP6bmTNA~o(rbJGRCYkOoax{Om)OmV13#VQ(>FfYXwtb zJb4W)`(CG*3KMvKGCOT|?86qOf|s;{sqjPg=B@Q-XP63ycSOfj*iZbo2~z?4lYyyl zJ#D+bIOCrZQ{kq3OE!IeOog@3;=dZE!Wz*5zhO*;j5|*qQ=$0c(}SsCr%Xe>o+75g zp*z}bfL{hv;T`gchNM|S@@-uQ_x?WwN*`?6wn8{ZgNkQM8cGjOto$KNh6 zlY99dTp8c2l{B8Lan91+%J|Og6Zn>8^Ih-4$8-HRBin=E5eRK2TH)cU`O95>$u+h&93 zhIs$n8gz7G->wTI$*eR)pXRtVzOeu-u^s!B+5; z=Q)3r8*9{iGT#@JbQye4(=$C@4<_(@`UK~__BiADEc3bRN|nE#HD3!D{`KGmHa(>!LZ=FlO^DK3C@j|wD<155v-CY-!CY4IKfo7E1Pe^db(+$t+!KlirVOT{3gF( zI(qtQp^|=D=-2e2XSX|W_aX3<7*9`@8fnbatqC$ux8-DHgX_bY&5yL|*`!53AMP1& zqyOF(1GC}8{LcW}l64V#xx+o+z?6WmB6!v^?iv_QE!Z49J0?1VJil*_J2a;yC-5}S zjq>or50@AA!=FZZQz_3FZ#6*w1$@G~e~NqYOBWwiBQ7IT|2N{sS?LKsMmU%73F+{a zUq<{GD;?oS2s=Bb!?(PG_@P!h!VeN2Vx|+HexqD`!xs=g@OZibx{vVLR+#5MCY(uF z&TbrrekOyPt8;R{P#Pcu&NF$C9Gc!LMJ`;h$Pxs)MS!#)D$pQI7` zw3+y-l2+^hyGj2beJSVY3!$4k&F6X0vwV9-@;GWM)jk3!&wUpj-L&fhRVM_4yP0e8o20s{_{O^I(0((T{=^V~o3DlS((v zp**)wtNs&t#8Jk>wA=bd=qK1d#3nT-O$!x5zYCz>6=pw3|1`5!5`3utm+>7GPaAV9EsRy_$>RA$TVTP^IA;^{un4}iUxP1Zg}*W9g1_%t zl4D>cd+aA-B^Q#fhkPXO?>fnsaHL(|12$4gKX>R&=@;rLygrx*$=wsgyGkz%XCZ1F+@il_?)kWoR1xv}ZLkoDO#_H)BJcDsBVf{Fx zobOH2@8d71ST%!pC&1?lFGgPQt=Q(lcE9kq^DmM!_Nsn?{Ve~S9qz!rZ@2?E)T{-4maE$g>f&OZdqcc&73GTWuTm$yiTi zj$Xq%GilrGFz#FiIEEs#(!iUQ_&G;x0S_42o>DDzANYLt%^|J@p22r1f3u}`!+1ON z0F4VC@0w-W_PcMyXE=rLETL10-ZY;3fsDoY6cuu@$8X{BYCXd^TY2y=3y=2&`090g z!#GXNYS zIGuNTQ@*?_<1TX4;6;Y*IKBSf@3a1)E{>$$RREP4c)UWd4#w6qlYRm(QTS^n-<1-+ zgRhTMzU6WhT~RO|B~I$-X-RN~^ZgFKMPYmz7xt7x@T4|)iPY12Zw+&5E8iuPd>hf% z;-t^v$)2%p_)47anF@Vf!`SSgzo&wuDm=P8UKc&_;QMVW4wbu{ZSXvWo*{V`aTPss zjy+y4R70UIS~eSN*?@RMpjB4c6Gw z46H$qnO?AM9LT{JKL}&oZT&z64Bk`b3dA?N0(?&`A+A{k{#WRlvYHt?k#j!o=M%So z5m+x%T;LSuG{{_PO>>0=b7TKFH~57Te_1#lz88S7J z?p1#t?Kb2y_IMeuPrxUAADf?GiK!G^B9-)IgHCn+9vuPI_Y|a&M7*tn>qbw;{VQ1qrZ*5 z1c$GHI#{?Vonhx)O&ZZ1|NeRyJI`kH-KVkhIOnu4^U_2;i)-=RME!Ege}Q#uWoi|bH6$Lx*GN~8XdDExy zyc6ua(WH^J!Kq;9oh2B2QSj^rOZic-^UgNPZ^zDi@19e~&g&(4Mk(tI$uj~w?^$T- zHojqXIvGYdU@M-M4Yw>FopN-bHa{SL>gQ~-J)&`z@9dddX-#M)V zS5U?Eu9EMEEP^(^4(vR1ll7T}gR!}Ub(4eJzB%l?>;H()Z3K4SPhL9_JI~x&ID1skTUNe8Qt zAwL_3oPeEI4h?)&*m-l97kOEy4?FMTUFh>su=AkneS)3$XZ*P$vGeAL-xs{4pO&`w zAo{(Aj&%<@(|*pjCk=snUu{Hf4(Oxxr7Bk*Oh z^TwV+S)T_xZ#wz?vtZ{yOvezwoiEqbbXI-}w`Be2&@Tt<=K(p|vXWIGH(2ed8Ht*Df z15`-f-^sV9SFKIfE7o4&9E&aQ3H0=e6g6CUWU!6Bw0?YFiuH@|NbyN_B=cQjcxB}> zbSKhe#;epyzNzC&nPTwFJ)~Vs8$CVLaCttF=VPUwlq0;cjps(*Cf{90o<=)(?rXWi z86^MZZ@T%mUT#3%?*dQG;Eb|5Csp?Bm*#lK8rXSr=>x&e`yKMo!p;kDFW7nGuZ)hJ zhu*tXc`rV0vs{O4`3*m@VVe8wOO@*a{TIXucAk-*vj=Mk3wE9n=6OBgd#rHNP<zne6Ioo9q8<9@=C*m*{Lv3wE9nri_~iM`Gt0@wJ!e6Nwk>JR?lI#}O9nJS+Tt!h)R_ z&S$hfmat&wS>Zgwf}LlC$?sCa|LoX#57Gwf?BM6Y&ik?X-q(ShS3+50v;P~qoM7j@ z$yKoPQkaup6?R@De!RlR9q?7b1!_J;Tp%yJ=oPK;uOD`A{OeOmQ>s4H;w$$ZOI&?N z$sG;-Y<*_R-Vfhv!{n5_XEQzwIg2yT%c(p6yeriA&N|PoMmjS2#+&A-dXo40m&`hv zyEx;#+wr4&d9(Jh`&e7P__YM>-Cl}kt?W2EezlMN+!gHE*06`$5lbyoz9F_LC-Crd zIe|m$pXRgP4q}H3o`D@>LE(cdcqdKkQ~$7;hkO^;$G-0>_I+#E<225K@;%+llkgSU z5%v{XW1a;)o!EJogPCJt=Z)q*8g`yj=nWaNT73BUp6MX%%t}=<b)?-x8oW9O~KC+v%0=k3PV_y4f-z8UPi z5mP&t6Q>J1?;i>6{P$F_^TIr6q{)NQ>R0`DI}iH2*m;ezRyYZE-odN>wXpM6^UYKX zJ5T368g|~#gWqoKJmk>&Oiy<`3)@$5;YEh6OtAC*4!=KD?7UIDo4$7BX~oVf?u3^9 zS+VopygWK~-hBA{sbJ^rz98aU&1uBW)31n*o#$apt@EjMF<`9?%?(tr?jCgZW&WzH z1ih*TpU1U)*Yt5M)U@dGC%udMJlTxxx?&T$=X}1g+mhrIn`_zbME$AOwiSZWCEtFn z<-7KBwy5a>m0x!f+@;9xav$Iu%|_@O|r>S;rT$-~I#6 z8$`p-o5^|1nSA5gKg9UH^|pcA#~#Pd!(VaVz|9w(6f0(_8fD|TMw zcz)Zk^K#L-qTybRAzrZay5hGd*m;p{kbR9Wi=9`FTZO3r-p83XxicwEGK61A@ z_Gzs_cz4jA5AT#LhF?U3f0C%#J^-O?N>n6(ez`OCih_H8FZv2wfitc!Ok=K z(|gVdbky<(b1(2DJpAj#&bv_dR%e^vNXK{E+6z60cgmGHo4@%)?7YC1cI>>DUW|^N zcg~p8kDX_<73{o^u~~L(Yc%XUW8O7|{b6>(*N5_bo{rdgMjEl5;J+}K=YpLV`QB#c z``N_9Yg6UB?P4oRqiz3(o!5??Hvt+D?7W5alV$(@qS$$7FgGKy^A5+huh)xAd+DiS z=S>_8Ps7h3@{C<)*m;J0R>O~D=heP&YS?-2qFbEmzBqhsgCT22ddF07W@KeEWpllFgI*m>_UFFy-A?{Q?iVCSvjDwuBdTzj)$^LeoI2GF)h`a6^GKLd8&D*SCv z1v~GD^vS7U=OI6~h zI9^{2#-Z3Q#1^?3eat80I5IqrLGG`kT#@S<=g$R$NOYG}WS#g8iT|ef5&0;qg0g&+ z;{anse7C*-kGwaJkE%K!xbK|_$z)~8LRcglLIN&T5R3+yNe}{V731EP1X`VdmDUxc zLKZE7!R0DS3$_HXH8V6V3aF_l3AmuK%F*qZ4zq4Q zqolC?s>|l(yBwq1Q!jfypYyDT=d#9pvUE?v$?Nt=UE=c;-)A(rcbDMFZtndJ+q?V| zNoRdJa_{nNNsqF3`G=AoW$*G2NSD3Kr=4cV(}mZP?;*2ZPxoHqGCg06b{cEg(PHOa z%6EyK$2@gsZ?W^*IWrX*JCFJ4hH3bQJo_B_ygFXtJPiI?iJgZXy0f6|!d_$N;iujC zResuH{KHo4JmzFOC3fBetj$|0Ap6AS%)e*7E4XY{vbN+oWFhM^o~>%| zvF&Pb*=xwfRvz|F%K5O>seKFUo}=sw$h5bSoqLS95Y^2_+=F&}F{0=4*=r+l3)aY9 zq7|mEjrJ^G+m+w{=CSjZkNnTX&Kpacx_teHvGelH{Ne-r-^9+#pq^;4^ZJpl>knez zS^A>}+4xOj=j|UAw#N5AjGgy=X#f9W?7VHn8<5y}tYv&IvGa8Mjx+YveX6Fv!hJQ- zV&|2d-yJ(|Cjb8%vGbk)u5TJU?-1v}WKVj&V#LZjgZ+JVLx`1UBfiNfV&$=>b4zRz zanMqIRT3*Nm-uqA#LBB7R-TVodB2bKF32@w<>kf|%DH#hpWHG?DU)<}?v%ni_8Rf= zl87VX9z=Y+-x42B&P&%C@$qiQHVw=u_6E`QR(w2ntfI%qJ3_k+|1#p^HE}k%p7?lc zy5i%d`}&8K-C)CoE`qzLE`uwG~)QZDRnxPvDvHzDa}RN z*~I2u&v?53mFS&+lMPpcZIwUQ=r^sM(k^Bx{YPfusxd<*FkN6&hHgZJ<_dU`th%%3BDjFpe~X5Pc& z=o#fN(i%yZIC@68BJEk;CB~ig{TbeKc$fXg;)A|k&Ybbc`)v9rzuz&xTXFOxE!|8L znb8$T4_-J8x|8qHhWtuQsK0f^(M$7%_+KjgL|&n_{^W0FeDpYa(AKS8arTTjdQykP z(UV{I4@_R2+0Cop#SRhP*5l}f@%A**C61n#x&?+JiK7R-yP<78j$W)W*V+h=OWoqT z8)(MT+o2j~SR{^un?2PNsLSxv8RuBf<-MLWobxh>mj^8`f|ey7fKH#fA4~kB1AY6r zKTTrIN&V92cIHiD*GL>aH#8U@B?j7B_*jpD#`mv_fp#8sj)y0XZ97kof#zVXWzhT! zo-dV{QYG;3*VNG!11-fBT&*T|gvUS&gvCI6>|4Y@yPgoL&Q5(5pqb;Uq?){KER3fXWLaiT6lo)i-UZ8EZD8vl`p`;mu>Zc5PQ;i7Y~ zG3o#8ap3;~o|(iz%cblHHP}wf3(*A?xk}k&Xn2qr4{bbiURfuMrt^0&Rx>O*%fK!@ zXv9O)>5g{?@8yHLGMlbCs zb`?sT+8kioF`}fRi%!}uShjmv05w>^+s zjxEqnnB2;x+~3K!85^{dX9v$KJV$sA^Bm$i$YaTjI_ef1q*UmNej!JiWW4-(%&>vJ zvn1{wnZeToqfGnIzyEhRt4|{LX z!NkHPjBmcs19)@uZuE7G84EDeH(@I}SmKk?uIO~>OEvL;tp3P++I_Amw`kw8d+#OA zgPbd=M8=A25Iad??%j#3c%GPhYlsO^ugBaQ=xfeXmTxBJ-d0ruy|tzP^O$>w!CP-J_s#&$ZyR&Z8jmgXO>9BSJ{29}f}g)2J&rb2 z@U7#Kn0qz6Tk%$Ui@7(SJTi}r1E%BP?sOe@r^eh<*#4TgKcc6uS zsWHyr8tezf;SA<7=1*Lwq^-tg(eo?FkO{`RL1ey1$Oqq8`GWBh<58{b1YZX#sCR3j z{VyAx(POhIPDT6ObanY1Hdp5&o7#Dp_9Yh8n#%-NX+G7#y_ss&^!doB@|x~=ZS!f5 zIF`s0vFjaO`R0;Octq06ixuteD~)nL##WK~G8u!{xLfg6p2JtXL>wEH*#8PK{1u0F#UQcYTs1tU7Oek-8gJ!{H?lP424!m{X3~+ z5#^x^eY|D-lx#&CkNHgGiD#6;l6=s&Tu#DGamwkgGcV7_o7yse|+7n`$`g86G?E-`CQQZ2HH1x|;rl zkB`uR_5S(~m9%!+6u5GT!A#ywx$#C_e=*^??^Jl20;)1rd?*rot2ek>s4Eh#X5g~g z7Fr31<+on}%QGe{6%nw+bC-qiJ@_2Yx*_q?;$`1WFe~UhxQ2eqc>$}vmoHV*a)?h_ zPOPzf(G9sup~QhYdhLtEdxA$6*t{q4gE?ZHj|g8g=LB!Xd~*>KYP&8|gop0gB*LI#$Luo3R!8LDuTurEl_G+PoZ?Dh6PK+DSt$NZQE-CA3`( z3@+kmz1(myaNUvX@)B4V-y(Ketbx;vaz$H+UqJi>Ux0r%Z8LtG9DN)fnU_dj!I$_G zBI}fX${a)L0vBcP&Xl?{e6y+VD0ZIcoZjloG4Z$)-e2@yPaYHcFCn(Wql_u%D}!U; zDdDH{FBKWh7@9KLqNS7wI+pU``C|39(tMj~D_kBX5tCZ__XpB{01T0|C-f!r_%vO= z+|fh7e0q_TP4zjJO+g;_{8l#6_@kPYa$Zi%COnD=uZ1;PziK z_hp`RO1>L<=KC}G?il9`{+ar8f2K>(ME7h)PZ8AZ0PU4E)XeCt3xzh-gFDN%eU);@ zWGt9-Iw)JtyisjmuqToawp=6fpqnpbdJGw#QJ|@VRQnuV? z({4;h4{Wn(*URrrRqHf$J zB~fAAzs?%7i#r3*czAnOoo!utCij}>-L5=Sdd>58SDw;d^Ss)XXKJr`UhK*c8=9~Ce23-eVxW5dB&xWwF?=sO_!xIR>kHxk2inMFn@E* z-wg9t{GQ$P&^s<4?1H^_T;AW6r*~ZbtSe9NxV*h9Pw%*#-Ib?zTvm`ra9IZ(i(fNb zURXFh$lP1x#u|&p$sgowpQJrO+9ljsI*ljXZV*}GzDMjgc#t?j=sQiR4s|-A!_&^E zP3)vt8NUd-^owad$8Qg9yN5Z0r6L1tlX}qk8k1)vCi3^_2m3hEBJ-d(bl#CX&;N#Y zbGu_Quk5A2&(fyEmelzfI3N2mv_f#xU-~FKPFfTFm)Ma>W||%=o^QI01efJ|6|E08 zpR`wH+K*R)DXW{qc%!d@bh=7hRS|;LED#{;Q*p znlNxb>@uurW1Rh#2GoP2vEJ)(;E=$IS z>#G|~nBr+Goc_v980@4!0}RJ+egWOCqT_AwF*e4680KTDKDS8wN3pjmSJ66S6)iK) z)b)x@`&_TX(Dl!O=eE7`4ISUFYI+gvmKO^>OrpLL>gDdQ87^XRzS7`KGvez@U82L@ zv)OcA{}1Xr%KTh(eRy33W?ko|316i9-eH{Y2j=^DqUqU|KN!3bE|V=<72M2+p6`6n z*?Ar|PW?7TyL6#(=SwO2)1f$)X5}acB!7d=rTfps1}sw?)w`Yn7QeNX-`_5C}&^j-QOc0+Id$feK1d*28Cq$uzU9#a48^lqk( zyNV5)<((Qu+icn_J4~CUnRYCj<(JryVzbnlZHv7k_>Ht#9x6L^ydvrAF!s6NE+0I) zp?RrC=%X527di-=7j@e{^P|)&wyu<647BwkPZJ-`hFNc#KAdKy%ZD=)K6#RP!RjY^`f#Ltyy7fOpxm^UT&;M1WhZ<39-fxd zPjm5K+FbO6ns#FDsq>l8cE>KkYha|JWyefzRDQawNd4&pMcFY^`2QgPAL9Qc{@3w; z3I8wV|5E;Y_+Q-UUuiY%!FoioWLUwgmwE3WnU8$VKb#X?eWou`Cys}w;uSn1o-mBu3O$STS zbBqZA+%(|_nNPRjQ`hrVPb=J^C}YFZQ(4cw0U0FwXpOp9TS!ngac9)$@2pMGn&G{+ zJVpDpt2>6s>sFjC<|J(&#m$x254n}M=y9-qOq{1i;$T%_U;LgpPj3A~d7J#P^! zYi|FP*3X#ZFS7ORkU1*%LhbsKn%pDC)@8)lIy2UYu~kKkt!Q<-x?*glv@$pH#8=Pg z90mLZ>Zs1wV>_Qq<^AdyW!!N3<)j~qGeNt2GqLUjcP6fykA^nu-^vp-^;vcKlTq+i zel%2Ohwm)h1;AbQ(a?Dtku@iYMJ>34CdSE}CtXpx@kibA?J;=V!nMp%q#ormSMXul zlJO3w5yo7*UezvIM(phh#%@P{4$S= zF9CXW>oPZ8m$@yb{M@GdWzv0FtP=`%zaz>wZ4}Wb-QMsMC`Y*7)R#R2vPJvH= z0T}VEC26%MLMsaK$sX&gG<9qT&o#|O<;3BCy~fp9kvOHXt+uvkFZZHWlr&FhtG&Ny zKl{t~k0-X6)Hk7OLOc6@{Pvn6*=Hp6&80rpFn5nzt(D8m#Y+ zSjD}3X~3UBn+_h?V{x2*h+TbbfU--M=gM&M4?j~5<9?8pH?EXA7AKvbZad1J#h8lW9!x6 z(Lbocso*BCF}aZY{+dF0Jhf^tA@|L}#- z%0|J#&8}ekN3mM=fwzmCB@Y$3OVrLtJP2*mvWFOYvbUi*jeU_x#B4~?$MhHY z^*i~mx4n3P(e`SAg*F%WGum9--)M7j-^RMNcJ4_-ug6zSD9>x2pwvEC6sQFsi{UZO zpmqRr$00SS);?6!ky_l;F(jsGG-Fkc{#M{=!D|ON4T00px}?IuD)75n{;R?LPpQFd z;U`m%XYikOWSzI#fKl{#8oUssUmKv86ZAW{5gw**vIn!R_T?faPHptpzEY&fy&Rl3 z39Kh>4)Ua}b{pT6MqNjQtyLYT4)I%$6&=gGCeTM2H#N@b&Bj;iME+LAI(1yP=c&QA zk?h-vRmLfCuEz3MwNXiBoCZ`)P~ylFtB`LpzcG#F7dB6*PLQ-djgA;aE2W+Qd&J!f z`*5rWAQSwFDcIww-Z`*Xl#S?88p<&wTRAH<44NQsAa$hXG z)|Cgn)DaA=5TE3SU3s`e`*>)D_(zN~N!oYKJpZurz_SmVHrz;TyLU)GF*o{rJ7iDU zMeGxN9imHY+FtbaK6{Ms1A8CehxS-s9R7qr33>s2TY+sk20UC!+y1y|jsCa+JeTmK z@|5!QhC$S`1K?5F$VrqlLdo~w9<@O+miljlmF zK|Gi9WbjPPkZ&TdnIEv`^RlgK`cd{p1?=1z z!n(4Q^Yi_2#$EO`=w&x$7-fIaUA9=t@_ZClwj6k*>|)A_O(=FBG;oUFRgyUQ2IGtEx!Z!oE@}>GF z(x%iQbL#Vwkgc_fR&Sg5PGDrQH{tBM-ESQB9UOJ*_Jgy}Id*WOn$Y2G^z5#D?~ezU z^m*f8<-H#tRMe@*UEqG+4Nmi}_za&4ZONKd9?u(N7^6IA(-CYy(cKC(aZL8tW6Q|i zp}p8Y%dwqgKd!M)KGkQn?_2!H?)?oL4o*vX_u$eW{#$Q<$&6p`Uh?zx2VehZ>%rH{ z+V%F^7(e9C=Q;JN7RVDDHr4mSePa77Y1OO=jf<(8ejc!7^P9u(ow82EdYr^m$d6}k zS)~*r(;s%6lhG9u>OAsFxg+l@-lJku);1UUhoFDh<0Iz_p|6K?pf3YvMcY@aYC>DH zN6=r{F9l8NbOk;SfcLsCySdv}vF77Ym*3$^V5(rOkAfQ;eJOUzePik1D8m=Sd_#0W zwfHwa?=CZmGQZ~AuXv=-LK~ZraSN$O(nH(ryElBV5*0v**7qm%gdWPurQVY_$(P-+R@C-Uev= z(Oi5Trc9E2Rs5ew9VZuDPTQ9O!=>oDOT56O&!OP6>DbEC^>&%VR>>T;gnNl6$k}+s zx0nB-C$8wLEMHSQv{NShkJ?+@_Tm#DkaOrFkVyrqobY#Md;;OXraPnUdqo~Ep<0f+PS zU(SWb$H3EBVLYAJlc!fBV|1R@%Ue8M;{6}-^els?U#&HGdKobTB6+%sv~SMS_}DpX zEHJQhzkqqm>rUD88#;b{l_6c1k|z|)EBMUZ(Y{lR9r0s4@!&rVl1 zZG>;iZLasQ88=-$NnNh{i?AJbn0&eh9ed|YcwwfqQ0O&0opPO5P)6(oSqHjffU7C; z`xL%6)PAqWnpHJ%B*hLCzMRauP90@sJxP07UH(3}_yzHs!rLrA5jyWf(jOtc{Di>= z4e*GcG-=cNzBI~r>1Sm7q6@6{N6^0LyBooqTiM-MHrm6Vb^UX%ebm^OnQ}xYyzJC?p_|EI*n8J1wZCZTu zX_W6@$2Uu9U-)LLX@3bmKcb%hCg0rlov-JcBJ|&XTHnm7HuOzF6uvp_zsNTOqVUZx zk96_nRFGS;;52@;_?#emK&w z-8v{Q>ziW79vQ6!H)1!{W4}3Ll}#VI)aBPPAImIZ53U`%ThtIIN7C!MTj^I{Drj_Pv60V5_jH^AFwIRiB--_cD}Cj}Pn|{}0+J z#kUp+eRgm)=eG7zpMsojm}%1jx2xV`tP9K8W5LHKhYdfB{NK<29yks3#OagZv?vUx zx=(>}|Das>IH|*oahl0E1xEL>N2Ns7D#j|v^t7+^LHp+xdlega7l}%qj&o=tK ziZnmx-`ZF&+e@B-=$5s^Tupm#R+>sQCF)TXwxxsR~MrhPF)304sEe=qNCH*-#l z^M}VSW}TCFX(uVH9h$gpjt9pipJd`tLIq@9!S;uh?ohKtyHY@Vm9->YgWbM2g} zKH7@2h?P@}uRDFHhqFrFf!C|vI?mHIOjd#o(>N0d+zz#%vmvANIeF(^(V&BUx`FMi z!`$xj28ag}coe@=s~Y@*Hov%Wh&Qx2g>zgnT78Z9G}x;^TgCOU+Ucp#FzJfQ9pkl& zi$eI39IVAVPGg-lnf*?3PL%bHc<9JF|59zvXZ-6_?G=1!;|_(!8h9M)s{yv{mnh4Z z!%sp>*M`v*7hLtEtD+%2cy+k&e|9%d)qfJwd37@7W>a4{ZMl*S+G=&G%PaZjM*avL zglrid>o_kfbguUipH&+;zYbfmlCcPGSG8>XW=5a!oy}6Usd6{_IjmogOz<|ynx%H3 zb_|>x%S-e+z>}k`j~T8U0*R)H!xNi`+APmb<~v#{{H#~w#Q%0x$aD*uz_=SyGc{_cBdti zPseF?AFaivct)Uy(tzU~{#)tdmytE$p?s4)YAw{!e6YKYN2#L<8u_JBmwrF5yvrDU zhWr)1CVPXW&uh>zR-exT&T~lj+wOfPlYWlk|1}N;x`VI!I)g%Yr<-|1rv09F{fp!D zbtmzSwlmJtBnBDd+&;RmH=c1Vs9=3IFTooKC3@TO0k+@Bn9jl%$@re2PT`#^8C#*5 zuJPrYKEC9W@lBQStu5B|RWq?Pr0hy!(#hHN??H1iw)$97M;z&qyBGP_+RZqI(2#GjN`6mb=T93 z?Hph&WnBK7w&$KR8ILeuM>OC5ub|EUfbF?6+_(QSbRxd}rHoaiuVW!;e(a}k-#&Zt zKL0kp{cidbzhk#=|8|j=5x)K7jDH>eG((1T`}W%obo&8s04AYP)1R@kILx=7Z1SP_ z_QU-dmr>>^zWtWRqPdr_qt@~+>0fDImZa+T<#E<3J6W?l#M-4{YsPy&y4J8YzY8sj zpWhE3-kzXr3godLWR^{9gYJvr?`t`K7s$I<=QZw%Ejv*gqgB_s8msN>Z+HRuzm9s= zP4}t}harzRcNExX)An$#-cR09gVEPFI)is1E6eS!_p-~BO=Bmj%dbKf+>Q^@ZTcWv zu2RYpXwN;X+lJl0LTTDRt50G3-_yLM2~q6WS?q@a*1gCCv11Pp1J+rSyX{!MFK1i= zE1XSr*s@)1eOD=Krrkln5#XK>H#VYwR`(v&&w*WNRUiLw8`o^lvT@%HYtM;I>F4`A z+RLJBfWClB^glHB%22HUyErg1-P@zicZX?#usX%Y+a6Y@ZsYNN7Ipd=mkQ|%xQuES zY`1A2<2wnJq<*DMHxL~i&)%Z^Me(cBk1X61n1bzb5}VboY;MQ*E&HOjvM&nTuI!G~ z>L%Hr{0#Uo>?zLN)32v^7uh8;rJeQL_AyFuB5UBal-bN4C%uo(;qXiOA(SBDopUUr^M*|Zqg2C@%B*349Xf5y8XT~tLqI;=w!wtty$D6)6X zfa6Tym?UuU-9b5Bf0?ixo~Q)#sGryXyAEF{_CT7iEvss}^zFy=t&YCQd|&jA4lgnt zc!NjZj`997kMkVVQ_AoD5@+yQ_MVi2e|_)SV@p@EP8sYtNMF8WJ^}7KSdUmloEPPn z&g12*pS9f#j}ZfBqT=XSSA<H;AuQFfqgC85X&16jU zGYZU={LE3rjz}f#QFye@k_Y1q9{tWm+!>6_Orx)Ecs;;<)>9e(I)NX3s_Viv#1(tW z*zY2Gx}Et!K6pM%+}DZfpbn8)&VkGo+4C&8d~Bg(Rp}zfsw_3V!_FOAg5xYz?Wp7b zA>=?Lj$dOxl7(aCNT}>MHg900swwOd3*5{cg5N6k9}R^5J|g~rpLHbRlYjExd7iRK z_92B5lmoM%A=xK*0-I87uiqfQOPO08V=f_ZhxnEKhjKz#`)f@jJbEQx@-SagSh*fZ|m6aN1p@aZ(lbA8^N|K z@DVoqtO3ZTSgpL4xe4P1Uv4Po4iR@h`g-kk-V?2qxwfzOM6DvU_0#fmnfK)=<1-VL zv9dQp_BU=OmQ4VfUJ5=>z^jXF=^eL^cy#4n);d<(2X^F+W}TREFPr0jJ}nd)>*_ny z^AB5s$9=aUbN4S+z2)2OS`&U|>>&MK;L6dLdCQa4l+o;0{-*cTnR}I+R@?f{k^LXx zdzJUH=Q(Ft?pZn934EsPS0<*ngFAt({mMD)h0f(p;LFUt&^gGRL(p5mZtPdqZGQMy z=8WqYzoG2y6j<9*k;A~q*c@oLFIl-?&y&zmldF{N$Br$9=EwE{htPc5{Gv{gTT+LO z@9c?xQRe2#{Gu)Iun%5v_&zeX4cZETo2c@Aon4!PkMHP6>SVqVV13az$2(Y`|CK;% z#R50Hy8}KdjX0-W&HP++9(*+Y2)r#e$n_Jj3xP}S&f0sEeDpYq`W;Fo#ehT`maErH)@)$!UQTYN`zCcH-eiSXK5 z{&%dvpD@32OZohQEd}ta#J6hU4rhTwY=qNU_@VmTe{UT3Mp8!RL(%H1;hXG1mvaZvbeG;o_PF@%=%#__jU9$x zpq#n?AaM9S%39-H53dOg_GPT6k^Vy-dGEvfWd2{zGlR!RU&W7oLe4TR?&~cTUO1#Q zg^!OlHrN;XSmUGKIV&5xnEGVQWxPLtA5IQ2WuM9KH&OzB2#z#v8PP^lybYEfhpjwnmQ}6g{?Y2;@d#_D0?XV(0YI1 z5ZnvTi|q1d7;8ZCzn%X=4>B(hmOjj=-ZRL;xmYl72ELh%bBRUu=S54 z+ZEP7Y}We6PILWZr@sEN&)C;v%^PomwgUUY_Vw^RmN9yaep+o*V?UPD-ZI+1OZqPK zYRPX||1j&c=7v{>)#)UDulTPHfQPx16}kL&gR%bc&MT_ybB+qTtbhCxpY`Li24dDP zd*eQRu^V0&G0-~m7_YGJt6%E=jz6#L-B@MQ4)l@OJ&t<^t;(oY@WF{5&gj?&?(Rar zzFtr_eTl<&TfALqdOue=upJ(s3QptWct2M;P{R8X_Tj`Q@?Ok$<|a#6CynQv&im8( zF7JbRPvw0k-+^mM4Ytfc-f!c(y#Ji{Axcx${mRojfNM~e(zJyB74+r)iDJH`$9oci zEg_FH;mj>`nN%L5O%$Bbzct`b2BUb7S}9i9`LSLi)*z5V#jQ@Zx5=r9>;Z98Y3k<|s0(3;4x)-aevZ%SY0 z0Mp;m4H9ogY@iPIR3@e;dN_AbrgA=hqLR^(4}PxgN@AVFIN>$UnSA=`8n8C_~)_9%06TKL;aPf*D@AWm2hMjwNbAwJ=5BmT-vr+Ti|oR;|FF@wvGnap zy>GHFHQje6^TJ3xS77(*X9mFY3-}VBB7Z;Hcba>orG42S{dMi%5#IjiM*C^Be;;KX z^i82}D)`XpsyRlx%G7cGT19)A_=YkEdy_jkTX{sj8U8kO3NZVD?e8ClR=mu&*Ll8f zKg2zh5&uXt>w?z)yDI!lcV2L+{K|m`@0e-qziYq`ZOI%-mwiHWDWfp9k$WCwuCp3A z?}RV!f{(0n)p9P1aut+oSh(;};rsZQs@=-z8LJ}eXfyZe39R+|lwCK1f9dxU>XrR^ zzj=zd2;lE`FLPGagwY|sa%9tcr2m$5zqxOWk2MLQB6QPt_MDEs;mc zNqeWjV*2F;jzZuNUcVJwOJ0HLaq5x(|E)dhIr-3Um>;_tnHRZ7y#t#05BRY;)7`yC zeZYLfZhLpqsebG?Nvnr$BK+7pzs`>xzGn}<*`V9e(7E`rtv!1!!Y>hi>{9rk4teM{ z^}ltVUbm<8vz^4h2PUD%NI3{kgzwjTINZN|HMU^3xpzI>zx^0>ea5#>c?^C_^TlGz z@8@08+w+=!d}6N3*(e(}gPpyBG3*iK9PNTw_ONr7wq%g*bc`618Zl>zK{QAgM0WUr?{j`-9Ss~eSvnV)KS$0^WaZ)agy(zK ze=2Mjgw?N~ccK2jQLmhL=~4Fl2-uL(qr=Ktu(eWlqdwmFHQ}>d^#0k5i}+i`*9eaB zDNhFP7lZ#vj6(_IQOsF2m$yCdwOb^XMqMm+;|z5B80CPS{f&<$IGa|1i`l7)Hbc$q zaFFJiz2N!EJVpD0InG4V0`z$)XuJ2H;4^r(T;xh1N!L* z!?6?HCi!mz$D-Gkb>YCF;{Z8ZCVdTn1JOAT@I4=$tm6cjr*?+Q=7UMM0`sP*akNwDw|#{QI-z{KXS%c<@W{Nk!72X*T#86 z%rXCxtBgG|qI3MSoXM5_D&q4O-&E-c?qEn&g5$wm9S=T>;0ex3&u7dg+A}*EPUlS4 zNNkWo{Ccw&bjlg)_0(rs2WSI*jmu2O=`uhRloFptIG^#pq=PLS4s z-z&&IkgeDzM;H%3zO@S(r$w|=I$SB6I8b@I^gL;2{+7TchVP7+wq?RYD&v$>-d8)x zetfyV`$*nYugrym%$0RtIel3pe78^E-|=vc(pSqGA@Tqi(g%4Y25tP+inpCPUHnf~ zZ$2=|I!MSixZ~J}6)VL*I}5y<>jV!M`#Ddm&$9>l@~}%*;R6&rIlygboq@ zEpayOyU^M6;iDH9gqTBoVUN>J2>pz4=|1`v(ap7Q6jfb;kA9*n80b)fflvs#V_lrQ z?W3Wwv$+#ZXt5&2pv93{-cX{@A#)x(bQmZ9@ri7Y(|&%JNr$ozf_IT&`}kIjO*U2N zaQl?*^)SBg?7(i`$2$KI=4u}qXYSr(e9AeiSjD;Sa?bWuaUQvxvx-%5oOk6cV^v%q zey8wj9l9ub8F+joYSv@=2$4&bTm)l6@Lf6#ytmee}U{dB7PxyrE?LqayR+_ zS`nCqUYxT`dT}OekIGq(1wt>Gp8Zd8UgmVgJBU3&4xyDrj9G2BKR%G#`JB85-iIb! z%&kVTA83v(DEG$%=h}jSJ*=RBdz3q9q0t25294z0 zacxymXdiWj?Fq`kFBy2hm-)%QSgm;!adm2ly;H+kRBZ2j_6S<%x*CQmo3nXWII|=) z@*`sUb$&tLY#++rtEID5ZOhQLu*T0O;__gR#Qx{q~<09xt;P>(?dP4ka z!V@w_68Z52&mj6$Xz#1XKG1nj&F~DlSMd(Lmwj;XTL2j#@q)f!UZC?Ha$wtsp|U%X zHxh$e<}8cyl&81Jo;#5@vC6K{?eSibDYB*^b0vu{B7EM4tQ*0Y{Tq4Uro71eO3L-n zhe}UDKE;m~u*c~>;qsBlzICze?_xht4*C=M zRwwI}_=t~`&<-*#fQ&Qrak8-wELuSY=)DPGq#KP59A$*T^@tn??8LKCRr!1Xy2;Sy<9dmVZC_H<|U`NL%uKec1QCeE%KapO!q>;c|}mpQJs- zyMvhOq7Th@v)b@X=W!Q0OKflAeD#Ps>vSn;y~Pkrqn@#`#@>x|&UxdX-*gvx^%#CF z|IeK#l)CJ*MDGl0Q6I@ZB{)f-y=iAE+Oz}rvX3^wldUGSs;9$~$o*r^ z-s3q7j{eM?TYL>)g`-UREcn4D*p;bf>iBsK{B)jzpO{^ecOLk0nD}u3n*im{8C%xkM($9w=(W;neo8hCLY*ljN>1Yukd`P z#xMOz|GusLXn(cZPmc%oHu1ndRa1Jz1G}8{^^_Q8{51ScYruE3x~+I%`|W*O6Z$83 zIzDkedP|P7YYA<{tMQDH^Q6F~EXz4733!ysIVa?QGXIwWPa|+O%J0dNJ-M7kyr9Y zkLm63l|x5aMa;>;y#qTa9=gfORu1H!Q@3&nbez>sIdC5TGiaAN>FzWh^y9n|AGo;zc}*P5r=jaI$7zZ-5PE(#7hkR|WBpqCF7v;`whYf|*5PG7 znGI}z+${cTC9MQHlKo0D4=m(=J$>2)A2)nivL~K>H2JEdbv6F}$5_7~s0{SzdyDkF zO&Pv%=qd+wNPSX%FZFDuURj$#M{|!z)$Y5eE8aU7N?iAzv?zEDkBf-Dc8nD-QOYCVh-s8eEEFa93;gqgQb+B5q4M_e4V5S*PIxVt$-8Ncm|!wz#~D?` zUDCN95q>3(dUyJd zNzat_b4*%2onPi9SCQs^=Tf6R+7F0)4zJhLb>EkIxr?a*S}NyWCTl)$E$QVoiCQ-M z-rU?Xq<_!t{(gm&V?6ZqSnN6z&Z&~l99d77y8+C0N+f--nNA#OHMnR`G5wk#=_Aba zQQi5+lU`n@RZPW2_;^BOl6VS9V%$50^G> zGt1uEU3QqH&otACgKy=}lsa?mS{rpPDo)_8(Qck`Uk)zNh3#>!XJmfoWPTT~e7;-$ z3tg2HBjgBY4nBbYMW}gSsXPJXZdF!`4H%1#s zKIyBDakh=($;F zA6m+WzVz=xQ~UM3j9vJUJQhAGSRX8}iPN42AKA3q1CQLn+Qk>#x9SMHyDr+@QyZmk z873WaHr#4pad-MJ^g3<$ zl#OwpQOf;+7veWH$I@YqCGF{Dp&^TQmh{qAt<=eRF7oU8?mOW9{bu|;z1`Qr{RH@* zH;Fz9mwA?qzaBkM4ximjj4%Dp-;|P0bnY%MyqJl7Gamh{-?f39a?~#JI#`3vEMeUm z-6k=!d#m@#2aS52;PC}u{i?c~q4#KY??5i(qsPnPclDE>nfE>P)V~uiif#)EtxLbE zXnz&$Tl899qiFfO4+r-ec(d}2r2pZ0m^UpSLY_J^kKjP)HI=?wunG=3(HTMTurMs2 zpZ@fgPx{2z8TVV5eM(_Wg@;~*e{P|kXk#gGre?dmcjqcv24lj0$^(M8K#egb$BMhh zgf$8sZ-O(?X)+Ipv|A2Lh>kbe$0T)2A3CYis$cqg44;eiRqB^MgxCL1e3~*}Uq&A; z6kLSGmlwV3z)qDi<&?E(d?#hZCb0Np#S~;zWI5=ztDLDDwoxvcZrCE}QFOzzl0E{z zlC&SD8?1JJuj8UyHb=sc^rRtIt^B8({42Q4HSdACoPM9=?!L&fbK_{+9CN+z^g+f$ z112f&ST?Cw`>SxNkw<0S#27;Z#v0%idXVzgw|n*a(tKwChkT3H<{jUl&7l!(-fY&J zLA|fgKh}we1Kl%Tzu3r!@oCDO)WJO9D0`dqF&CRMMhPm6eLc8u0#|OvpE-agyyn0T zcE6)|-vK|7c5^y*v&2#LqZ_s+826}&56CZfzSsBH(t+hH-d(2cEOh(&`QlGVCr*0R z^oCruFuk9v(2?dUEKnYv9$%--Ke9k6#HU@@LVJFER%v@eE2QpnXx+bn`yC!-tblI^ za}kl1?gY*`e5rVO-$aZ)JtkTev3%=n3h~an=jGr;Uk}z{bN1IheTn<-1XkuqO8szc zFEAJT`)SpT`(eIiXAF06zlXHxh#0@A)KeER zeszCSB+h7$IZp;-_r5uH??)NC--3gJh_SnYvCC)N1YQ}tjEJ#2dLd&skvK`v{GtuY za;aAzw+j_*1-KHp+>D#RT*bK6F>X%AZ8i7`mwoiz8Y^O*_19{FrJ1u!<~Z%N#%ZZJ zPC{4G=FT4}%dv;Y%Kw?PFLy}FHwzzv2P;ir%Vs<{kE8tr*ko`0QR?fFU+9UxIvEqA zjSSxbbdc1)nYP$NtG6|Uwqy?YZ_58~c<*o2?T#3`Fy7k`K4<+3-rFg>Hx5{2A1A!` zu*_Qp{{7>>1@GAv?YZ}R^4R+ePO+;x!BJwc{^dLM-jC;6Z&{lYvoq zFxuFr@-31F|7G%7I1j2O4=yHuuRLgXHaXO+b7gL?2oKUX?#0q#BY5y0eQeWwFVoKL z)E6qT5!b8_`wU{)ZxDxH!NJ`pHqP?ce8=EJk)N_&+$MKhDg(6L4bTB;!k?wF299<> z=dVNOyl;Zvf`?`(USchKkwZI`=6?E~Jl2IY`)RDLWXT@kmy7(wFWvzi)b%lVWe4S& zsYm4VmB1+9!sYXhe-RJT?ko(XVn=c&DS&SxaLCTR3aP%yYYqKW!93%A?#M33Mtqii zPXXF$7@%|p4wZB^Q2)sVrSx$MeVxpnii^GUy_*NKQ?<1(8fyhT#;SaPE-xGSCbG=v z(*SJ+?=p4)`sk=mHP?+5{cLb3q!NQz_=`b4+8%f;1=IeU&cdt zSMrKqs4sX3&}V%v!MM({#+5achndR>UG=D!K3Xza>iwL0#h#nWSa$-$w;k(i!^XNU zVytC;+zxDzKM8& zP13j5*BE2|0Wn#18ElMslfQ5Cg!x_Hb7WGwaVa>Tkb5Z1e8BwRVeb4u8l!Snrwk zghpSTg6~M1{r_cO*Gs_K!zX3V6P}m!D88;|NdMpXy4Fg)QG8ubkS@NiAF-z_ny+iQ zl#7@Ph)?c8NoTIwAmbb6>#CFVDD#;bNgrY6AJsjd`Wfls>$;E_W#a34f|wl9d|kKc z`C_yS1y1}wJ7_o3*HtNPMDcaalJuEo+xRW4x!m`qPJCS>sPli~>$-rmTi?dl<)S^2 zM|+s3OP{|amX!2)27R_{U+Y(VUE+g?=Ic7UmwpyVfADoRQqGd?mal6MG+(pj)O)~M zus7u#Ng8|ErS0XpigvqsN7Q575heECMYCAj)@{3V-_Ay3&qL&WU1Cp3xn|}ik5h-} zlE%TZ9>Lui*n(2eGr&**4B@=@pZ2%?g>x<0^uv-dvR2{#mj1TC7J~OE{C$v5}6HTKfhQ&K1Xw$admj6)fCw}bZw zCnFCdEu^N~&N8fT9+7?pH=4Tf>G@S3xjd0s@m7REj@-_Mrx91Fu< zN%hS!VLKDp{_#Rjn&{2<@>A*8%x)OY2Un5(aYXUG%o03BkyYQ9^eD3GT1iJ%1w^)m z$ttVwR}@C{DFUa_U?}bdh9XJ7$ygV!L`Hqpy7)Pg_bii!&h4h5(UN|FnSN1s`stEx z(cHA|^ejn-M)m&PK)R`Wto{v*=%4gQ=)(yeu_d(1T1NqRm;=u&VSm6h)`@RX z(^@6&epU6f<7>gi9i+{U<^0buY@UL|RaK)BSLJhWysdwH>nrf#9o(g6;acFWAg_bG zRk6fA(DNm3i#C?j^GK5H^|2A#6XZn(n0ygvNw_VnD*8rz~Utit+mNHLJW}n<+=S@LBD;P^ zx%E6z$H{J-fjDKHDypVWlyTx$#wlLLDNpt?r~6iXFJhc(lZdzGo8uIbCw!ca zEtdF(a&DoQaRP=R65kelUu}&EZU2igk#YKNSlXo#?(IZlr zkdvLRDSkzt)9dfbz^?a584x~4k-5kH>{*PQd%VFKoXkBoV@rySXZg4-KHfIIdoCjL zIoba_3|K6EXyt#NdPHa698D)i%zy2=>7I* zcH^LVu>L=vy+{e*=90~;c3%?OdT{;VHx7;@zCy$d7BP)Om8`unEmgA z4${~wBXLj;IMvlFrC#L>XYi_vqv+-c_{a1D|J_mGzs-a{2lxfPQJD|!_C48jaCSlK z!KEL5uHQQnzhK$!sjvU;;M6%S2Mb>KLce#U0{LK#v%n>~%+{Yh?&y%nJ4=PPzfIk3 z$R{_l+nT=$ZG1VjE<6fP3NM8BO{XvF zRjI%6dBexj#X}|^d_;LE_qf@w$o3wKXz%Z4{-xwEq~GGRJ{)nkvr@IaNUg#bNh~1; zbQEAe{T|9G*e}x0*ZJRU_LIF(jY^Ed9dpoIKjoA$zSC;oTqApdvX)FT`fIGySZg%c zHPcnv`DtW-&GPF__-`=#D|W!IxML!+-`7U8`zy2DrIh=Ex{vdS-BmG3Vo?YVZ1{M& z<9a*iCQSb>{-B*XnInk-A#=QV-&%D1y9?2!oHx?HoiS2;MH}w(pQicAvu_r*1bOb^ zOqTw=)cjsPOq*4sXxsR9JKy@v6JL^{53KbL@j+TXX4xAUU*Pn%ueW<0{qfgrANqX0 z#3o%cMyt=8;FUA>tj~O2;2NwojJ(L(ut(VyFSemp$ooWZ0GJ%r6SO6?C-JBnwkx|T z6)tAcmT<|C7^enkAycnXb zlMBFQEo;-@vA#B|vmJOwS7d44JY`n`Nc*TcAG|w!g3}-JYytPeAGD#%?%wtm4+MWGolR-=$ab;ku~*y2EM1w1t$Mwh zhz&c?^NDRx$DuI~tei-EsA=acTX~(FJ;y!^k?uv-iEp4cAHjJux_tz%c`YA7_j;ps zr@8R&k`(qm3k#6fQLLs@sS&T2;&^(OYn!t&IjQ4BFbIJ6U}~;J;slLw^Z;Z`T;%_eqk zj_iFo>Dq&>e<1z-)hjDlFL9$Y9ID#7mA$BSoENOfAdPvkmFDJ*p?gG$w}3uX4pY#T z?9T_rO8k2T@eWT;%Zx4A+1bP7jOGW#vfjb`cn;sPT*`s9JPTmY75W2pSqmX^~rOq2iLg%RaE@Jh+B*ED^iMw=A>NC ze&TYzea5#cen028p5Fufo-=dMRc}>a?rT2z-92xq%BZ)}`;Iz$aKBNXsL7+a19?=V znt3k!kZ)ig@(o+r7g$2<;~LfZVs~53r?ge`K=GOKUD_)*+p9UvDDyvQvzC34m*IoI zoPIA;Rc*gL+4nv)QNX!HAL}tQ_+JUWBs~{A-4dVdxepktmlU5_OS@IT@F}p|?`kfJ z0e??^F#Oj0!OffC@IG+(Z<}r%fQhx6G!X`AGJ(> zGS6KGj@Yw(!)kC;3qQ%aT@o@Op7XqO;uAa-z|-#o60{S&S}c1Inc-)`ag&MOTQPfmle7#yV^C{BmA=lc=OqZeiuBGMI074--zwOc`b*p z5}uJflHZ+yZXax{qe9r>-O@|$f8uAh7K0JJTN)kvxEFz zzT5D>_>ol-C;CO?&|#a>6dIru?%)ibw0j3_t|qU+fD)D<2u&2fSTbtL7eJC=a7+CJi72iZp zxoz1YKT2ji(}^ELzMK0qo?&z$@y)}>@nPO?N+&;O0M?QJhl89imJ<*9C(!p2Gw))4 z_i|@Zv5p(DV>5gcpH-LZY4ppoZ7(2Aa3?W6YteZz=u5%vqQ{LsMar@E_d;cY6D$9H zl#%?moB3bkT%OoBmTi-KzSApb5(@I2-gC}yd1t8oJFckKR-VDV@Snq@#4KblV%2UB z>zMMbn7y6(gPcL()kD?($w5Xd9feuyhWYT#u|N3u9B7nyaKbV`$`*f7DnRI$hrV`5c{OR@oJI> zerFEhz+NdKPL8VE5COSoQ5_X^of=wxILA4Pv0={uRC#^9Kibt`g1*ft)j0YYaBc? z;KBM)oC`+AxX>j^cFr*NxRqU^I-5SBKg(>%o(END@WDPxFqtz>)#Pd9EQ{nDPQC~D z&by|cQOAc?$+?TI#2YNnC06a}_`t7zjQfHeLmc;}URC+!l9j?@QMn#cpeLD5@t*adhNz7Pufu^;Fv8TMIZlGV1WcJ(sA&PErzE zA7>o?l#=L6B<|*=O1$Tm1u0#zNgsZTi=2nUrTZL2HDlt0`mVYCEoW1`y3^f8wXC)mZWf=3cN^Pr@A~Vm)=v^ z>6v{~r{|u6&ZVcjD<_r=&jQX*zlxnXi~EZ|;+K0Rv>ULA zi3@O`oVz8yRQr40x2Ja0a9mT_mS2X*FTdHUOSRaEWu3xWwqh#62iG16J zJ<*{K@+{(iO*S@LO1!5f7X3E+o-NtP>kput8Bzz|1ZL>G3+6WxWsFlh)-oQC$3xpT z@=+gwtUcxE5|6E)r-EPUV{7r!mD600uhiSYE;*{&J(53{vMW*&J#BiMH}`0B?NI8X z%?h*4OU&|nvxsjXun$to{zd7#f#80psJgSWh!>Jx=|C6X{0`@^m%dXHt(RB}s=(@YDgn zUd%UvYcFM&0ZSFI$hYCh)ET7DBBs)D#$NKQr4QKHUHG){T%)HOc$TuSd=?_c$4#B1 z;S&#ZF-j$#1U#FiDjnK;5gcPg zT`zCeCHO8#PV|f-pO5^kYcypeU);bX{oV#2FQLCn?{Rl7rN2w=$?uey#0ok%p79p` zmiN@^B`X7q>sD@NEns=rAnKAW^eJ?%Nt%Dc*r)XP3YT5}(>K4-R` z)<{~u)vj{{cLs|clHVoz;3kl1Zm|lJ>gto<I#=PAs z`v7S>EST!M+g}`>4y;!D)m9yTnco}r{>(~imU@%){@-n-ISWWL@N}D%mfc4GHo``NX< zixscfCNq)8#=dD)UxSkU)cG;Wt{Lb-7y4Xi6uo*tK@K#W!UyT!iVtl0)BzuO@41U7 zyR+e*ah)Of;4aP<%6W!k*oMM;Hu#g+83*E#bz(=frw}8AwhcRCV01g;wBGHAkH5x_ z_$17Z_`3dn8{V{`WA*;Rn|goY&F%De2z?ra?{jb*=cMVg^gW*RfIZ)rX?J+W=Wx!# z=I}_2BJ}r_SyPDpe=JlsC_@=*50&g`%dmT9esA!qEW3JOCgb8qR_74&TI6Bi+SP-L z_5ZoU+WF6!-)Y&|xx@DJAAiR*^ukX|`&Fac+2cN-ZlyyR*!l(MsS|1c$tmhC*@yVA zB#9S3u=Vfw2HJt6JdXNJS@$7x%Xa#+f3~}GKmBQ+osSL~sOu0vI%Gd%B|79|bcpEe z|Jynwv!fW<-HuM!PoG7{cFt)YBId5l4a&Ua9|iq)zzdJ%DqgoO z(X(U}cZCfQy>=-w>=IQ#KZ364a~xiJ5mX@$y_KKeX&o`UgW-UBX6EDZz8XQd1^@VTY3F?dfsQq zyMyx;jy2Kqw(u=S?&zT2r^qWZdz+lcaOUYU`zUGpyWf!E2YJtrajmbwHWL3T^R7+c z5Wlk8v5hqG!^JD?N0{gCgja*{^YT08&WeKX)U6ziO>#+a(Z)s0r;4#TF5_(K@6d^> zd6%>G$w`TxdiXafDali9^6@R$NC`;^olj`jf$YNY3!PIlCiqsKnpG7?#Nz z-d)V0Z(+~E3hEXAZ#BOayrYLUiGTPQGIJyU@8azKOwz49*BN;>>$p`$YgLpJpG0!_ z7}U$$SMDXr@WqiXuslY-jVD8!CI5zk-eR)jOGx!2)o5ZL` zj@MAOP=_Ve7ubjn-X6pHOMl7@VBH`_vuCrF!wtvOTc*qG>jfY^DT+^huuMBc`nGqg|sknH_rpQ^n@p10cM zJ;nR!ANqPj8y#LLSG9Q?_vbAs3OtqI?HE0{pni(7t1Zu_1%D^_R<$GXa);J*c=Aft zE@E^ZH1<}g+NXRIf6s5h!`0yU>u?df{y&e4e*x!z8yC--xOnC(aIxOR#Vl|!Kybl2 zz{v&B6F7w4H#k_|s?VF~)#XXc8RorwtZ$Vly9yXX*+<{b83TQN%dTiYzC+fFCL#9> zUr(r3Vt&UX6BCe)iHrv}-hyQBN#vrO-H?4n##)~W&D(s}3$Fj_^d8pZF&k&URrv0( z()nyAyrA<=)kW|~Ci&BR7jdWjSoZeRu?OgU;E}bE9DeWRy&k^|F*$tVlNzsoGw%D5 zSQhi>Z-Cz;@EUgy&>ns&=>C$K#u|^Tdt}O8tF_Rz9iJ?I$y(shY1x0Kc5B%9t*36C zmN%abE!({w?57T7ax1hv2l_sf?;X$e@h0l>I?mg{-7cYxinkRyE^iV#{(sne^Z2N% ztMUKynaMJf1p?Us5he*r0*DK;X_QHV8Wxu#RqIj)(3%i8s#Xz&EGU8CG7z;L`jh}J znSlVdph+t=f!2n^1;oc%zx~w&wI)PGszRb@e(&>HG7A$7)aQA=zvuP)Bd^Kl^SR5p zXS?T~d+xa<@Ug}F>FAiOs%RBHR@|3K`_k_K-Q6RZ|JlW@kUK%o1rj6|EBgwEdd?})e?IM3BgX}t<$8rb^u>-+^orPE@~#nC zww0XY-b0ytL*M`E9CthKCI5e@Jm1(?xJyn=e#X;&qXD-D<4xzfvw(RfumsZag8yPT z5<}q_JkGrbI7WrRNorqVX*f0B{@h3W0PDy5qLi*SgMLEO*_@S?_`kM5e5qsHcJz6{ zbDT-0%y#OT`$p-M`;eH(Zx>(Ayf%~hZ3gq)bZq}=Ugo_Xn>e$ivKE;tvVu z2a!2Q>`9p;$#3xaJEb%{C?H^az<}^>P_fwqwKXj_t3id z9QP}XzfU^rWM%d*VuLB}U(&vbBhlF=ju(JiZTv^a}C>_7lGd{7XFTd&*S0ue(-&N$|qniM!1Df&u1Mtn>#&(Z->CM*JD5B8+J@S zdpzdGl@u5Y65)d3K^rx3{|O z-i{gATjQ*Ldn-}SfJS0(_0JNUuXQ(Tdar3`B>Ae~S%Ke#rIB2+=Kq=O+nE2Cvo|9= z#u?!4smzh)`-zTV*gv4{u=4f;!Tz!VyU3ab19RkD_IN~3v}XXLoC{{&)9m$;hTou@ zocS4?bCz?_;P>~mQ7t|SY}gg(JBeXyd!BRCso2HfM`G93!!P#^)91lk$%Ad`Nv+<) z6SaFz|E|lp*5B>Vl2GvQ{xdp_{@8{OA_P22b+`)#3Lk^6#{5c(Us z{4{Xy#rLhr9W*_W-2I)l`XYBvbA~;L#)kc*>+7q~!&jn@uRt$fj((nr?{NleaNX}A zdY}g$Y^D>zQ`<*)?gpNhxI;|7>btIn`)AM>H-U@s+!<3f zTH)S2S)ZqB>+@go>|%Z1iJq19xvbqaTC%>}XyQ&CcLG?)xgYH#togutT-(I;_Kt^f z?qBq=?Q?}L@Ip0yufb{h9MHx8!8kt=4~%o)(8s<~=u2MaVo(RKs#7~f5MwBb4YOT@i?N$k9t zOK&|s{z^-ae`#-SfPd+^&ANZQJu#=YaG+A#PTq&YDDpnw19g>_WJ^v7S7|A~4cW2| zbditc8SpGVsSid4`=s9GSs}k+Z$7F0jJwAtb*&1F%)P+=+)aL;R4RFejt>52wS3uv=ektt}!@OOSw`k;TGW0d@ z@HhMv#I(;AKc63_RNu0`!i3$}G3WbXIvNC1s_!P+Z^gb9xs-T7kwxLl7Gfb=qKhxW zXE{q~xRx`X#2KCc3wWnIV9j|CF@TbHL6aZs2a(k#>X`BqxGoKq=fE(YziHHo2n zh|HafbfrxZ(S8b|eUADZT{S*T$lb z*7Sv(PY4{#+g;r|_Ms!pb)(?m%P#Ie0Kek%6+IHg_?iqo5^jC{U6xWw?%0VE|F8S~ z754?;Li9^DbJMlR^@(+iwP)=u`s0eP{raWB&@W;Gihg-|p3_@Ge6Fr%2KlNv^B$^W zT3N5ZZS8zf?(^A&&8_R0A=-Ez-E{Igri8w8SR)k^>nr%^jm9r$>AI*Md|W^|(?7n0 zXYL@}&P6HO{HxE+GIuA#$4S0z;OHx4PV`Y*fG#S#(XWeM(tII){k@(xgY*-6Le_AB z`l$;28@LwwA@$P(qT59u^rD|i`r7v?>Zj{z|77*kjeUjb6!p`1+V6{gx|>`Y|DF0N z1zr*T^ke$%jdx|QIh=m#-&dTSqJC;ae*azl^pDIF>!)}4K8XA1CDu^%lgM)eHiIe8 zZ}44o)kEkh`5r23uWH{pw+{ZK+})H*M~|FnzfJrz6Y$Tx$epkHd2yA!YR)rpr*6+a z7qXMjU1B!FXHy-|J&3@TI)Hs$ehc^=jGson*YUr|n&GF}rnt^^i|z}N@4fn(Kgj;r zRs=t5`(hKr&L4ZVHNRnle9HVGywZvuE}?yMe_ZBTSwAoad@%Sd$c3ouVEh$RBFf`!0`Q7rrOn*yX#yoxpG?cnt+RSo~=R6K~3-Jy+M8nDazTza!^hcZ= zPO_zagI-?ZeawC@>1)3|evp&vcNpypZ(MswUk7XB$A%I6WhCPlKd1JNHHfS~#K&pc ztR{>@`wAnnd@B6%)ie6KS%U+=NRvUcLe?fH^4o{n`*LqwOLR_+>k*~qV1~160q1hW z-e)b)^_i@-gom)#iJ8#O?6g{QYUY-5Mwf9*Ig{TCD1U(R|I>2%NAU7rDW^}<{>jLx zg}jNUBBzVVyZ%GT>7u^k&wSd{*Da{Fz9c6S1mYcM}JuV9$?xgFI{6|MeGX_x2HAXj8~9DQCh}?yR&sYg#Lm zn$|V-*$37%WY1k)oW0bdbS+I+CeK?<>_=H{r=0b3^(($8Q}ji;|BRd&N2BS>`tj}! z+#Rp|-!jFW`mD_N60gIZxWwB?434H}8m71lX-CebR}-g0ELP9Cbq%KV7y9FP%7Wu~ zI49+Put;Kf$Vpme#_-7ba_)IyUX=6Yc5sjd4o2}eL(O?>j;b79uw$KP4(AG15u)2^H1{u8)Z=D46IzhgRjSo~2^ZVvfhRMu2I z+f(L2v&~6vIpf*$zS?|$VNYMXs5|FQ8B-2p($BGno|%_1oxym#oKy00etr+|j$WVb z_GT#i7Es?jKhJnRPgA%@ncw+)InPA7R~^~z=Xw74eDO1pyOOcVnZQr)2tGR>ly89k z?A#kL>MW=C;8v?Qi<|>qat=VNC$jKOR*%$k4g}Ipat^dx{5c0STJ|C5fK8*PKj(nN zcm?L42*=Y)UjOgo=})<5+~nzb uxG7<6sgTb_*0!-{VoR|W2p6v@wlVN)cWMyt zDSKWE<-E@Dv-Dip!^d;aCH}p+tdsHcZFjLwmpy)Rs`T8o(aIWCzDu9-UHbg&4DzIK z-c?1eU7U}VccJIzKRQE6x+guy%bFLuC;7BHA&7@ffro}=$o!Y$`-Zu`fVLZH+ZJi7 zkX&d|R`Th5!ne>iWt>_JDd!p_*WtVKr0o=64DB7kk19C^dTXx%n8inOmshPEj-CjI zFF6k>xdwEYIoCixRR+w5)BP})W7p-g9`VB*{?3GFz(HTf@Hg5w#~|$o<3w-&31bXX z7{h1qh3Hqo$pQ3^ZJE<6@^#gnYEt0yS$C;Pf%1$jPZmApL-$P28RM={l!Tdug_DZx zaTP9g_&at(#x>pa@`Z+OnxM;a)dXT;Ij@M^OJ4_|Q#EqD8lJHy=(<6xx9wuRUe#px zZ23kV(Hq*gp~g4IB=^REc8aB3ae#hFn-osJ>`kl9*7Zxhp!AOi3u&Y>iBZ$^z%o?ZNe!$ZOo;U;nUBj z&sE&HiwZlNm`|p+UhK}{f8oDm#uGGeaCc2;o*Lru+20?~lcz?Dy$;Az!<@moRL@hx zoI#x0_vfiuTqv;m=ZxO+zI;KVH14e{g(qv&tD` zi!!)E_NtDsZ^K$iyGORfGN_`|GMGE(l6?9;R*LUqzKMRC%eQ&jx5S=rEqoJQFKY_s zg`FkFnN~SdG*A1#_cKKW11q&NMM^@sCp7oK*SG#zyC*T9b7jLh&w%|iTh5;4pc92g zHqKg4<19c7=cH}W`BPxnBxl7NhQB-=+eNcqIA34I*#y=P*75pGq9(HyNY-D#e|T3|O3wVRC*-&gkaxf4Y`$ zBxBe|JlmLi1O_>8kw#l`e`#QuKQWHbGFHmSSjFa&cAur4OwO9y87pn=i&30)g5%NH zL$S*gIk&5wU6pUNwa?bwy+d%5sW^Wt_kG$H4Zqr&Jbab~U9Hk~cU}~6LW%4%B(dL+ z%)Ubk`wywCw+9g?6y=t4K8 zxBg~j=XP}GADz^h?cBKm+$x-Bl347xnBrM0IUie;v&s|A^P{=H@oXw6s@+_$pi|DI zY>uFftCYH`*~-p8IEjC8sO}9jojW%s$(iDeVe!a*EIGjzba$vx)o%Tk@5<4KtOUJ6wvpwguZU+qv_Zq2#M@j2IS=uc#kq?V4z#Cg(9v6wh07 zf9a_yxV`qzw6TRYw#30Vla#t=vz47MRw-^Sr*v<;kv7KYZ6uJp%N|=i>#Uq7JPElg zJo`AWzKwhiw9z>N*{DfUy0%@X)ID>fvh$^S#a*XRf2wombLUI_Glva`DxRg-if1J_ z-JWxEi#=AJTXKKpd7)roZ4@}Bt#g;jpr*n_fjXCI(PA#z;BD=oMCqGcuvkE z^mkY7U3pJ>qJeoY_}m8WUKpg*JwH_0`TJtUy={c#t~rN$MvG?+V7#MqAM?DC=#fOBKKPRAL;27uqsD0gn|fwS&EWZY@H~co%4q*J`gww1 z7yZ1(coOn{ML)OICQxq-_}k3bH$vBKQ z=MObJLO*xb-d>OFz~fsD8un-G9|~^MM+`fA_e#%)l;1dYs<;2HGEWq=j0V=d^~<#} zZH11eY*ZoBNsRr8+;Y$Eyk*+=`Ow1(tTx_nESTT9jd5%Pr;+gP#!PU>m^MAKk5lI(>b%MC$NW~o16St#((@7Zgpd1!$3w`C;9wQHV>7thYUqh#^kP5w<&B)> znrs|GCkP&{%z4^#2t02^o=q9K13Wj-zLPp*8-C;Y3+3*pcYCbR{;v9RPXuEZeIhg$ zxm6ka7I@%!=r8(I>b326dTq$;CTFGRF6tLCeoY74z*n52$;AL@W<%C($YN36Q{cCt zb{=IL;eAbKBk$Y5*9P#_Og}DQ>fm=SzoO?m&~0-muQ0Ay^o_tL`uYv{bPMCz1pU{e zXN4BA(7D-(PKMvNQKp4@qrn|;b!~#@w}Im-a9j*d-$Z|W!1w;}#pB3`;Bz{;HLjv8 z@N{iOmuy6LZq#)4Im1>WZ+VpWu|D>jCx9530&ro+2a-u@N{@|l6r^HhYE;cj%Y6p6HgtD`dI+4^7{nfbZX^(|E)k)5- z*U=}LyNW%61Hq@%t43Z#uSj{3gAK@zgL;|JQk8lmhzZ|S?zy9WnP(U{6J5ClT0TY} zf2PhA{JzR>H8fohO{di!tuL6IY;b$LXLTBt;KtDP3_=HoH&{pGu#(sZlH=VthEBmB7e{9Nz!hE*OPGIpC$ zUf}-(_)DbR%X!`k_&*N56g+JP_Bf-QBJJ*8;n@V7GM-|}+ZrDCtg8QwRu-CfZH9OA zDX#*D$W!$$kEaS=Rw$d_@R%nC-6y!MHt<BU+B6PWFKdJE4V0bu_ z`S>j7uRL$$F6*Jw7V2+-Hlja8AG}noxT_7j<(UzpZ$}I}4W3AX?{3el^gIr~evaN2 z*|kPN=YrciYeXNy7otmxz}L+N&uz+m*t6C7q~~+S+R9ip9%8K93|O~P_Cv~w-u;MQ zk=wJ;YkQeZ80h{pDR_9aN7+M%Zv*=zK z1F|Xh!dUddpD6zv`njI6{g_|QZqV%{nFq$~((N+zX5BN;cMI)`4$x$ux#`fZW!k(U zbIlfL@GLaglAY_Vffog@!gqJ%m3i)BtS^9fnIFV9FzqSPuYz}(D`mcTg5M~99q5(> zaCt8CkkmPoI-9Y-g;%zLOREo0YTSD+BG!?zN6}wA#K;4y~s zGB+rcAGO=#slYxH9VWVH6Z4*}p~6!gpwC93dn~LqN3lN6XH7a>wRBj{kbGa3j&@=@ zRo1k!mdwjgW*=bh_7!p+eMHWZmPfm1e?YzJI7`KJe6gDpaw75knKjb)8tdFD*7ZfK zr)6y~dGlngE-=gu$N?uh@-uSY)F}Nbwz96@%DSHXc-p#teZabY3*Tj3{}q0ZBgCs7 zWo*6W5me@e*Y_e=>-;hMGO52ZKS`4IZw90;_NEop6?=y z(zRegR_8(LPa_wXtDkdcJbr-N$e}WqI^lAtToI6m>xz&(T-rDACAmV(Z>w@7A4Fgt zE?Y<*t{C!g;bVcOrNHO}MrmgY?K}!!Uw&L3F6sy5;mRiuR}AHRl($1;g*;qy$uTiS z8QL)~QJKA{pITACc(bhBZOVARChjeNxzgEYP4oSloHv5I1Nfajzz;7tYy^i#Df5MO zi0?7zTS(m*v@f`tMjw5}mCT{X;VK}n%5k^~n^)z;xH9_)!c|aS6+f;V=#Fr>D$w$( z4DGNG7nF*RN5;E=+$swgZy~u=#J>cdn(xL>GaY;gzFMtAwRqTX7hi|(=URL~*We5K zF+QQ$oK3izvk6h|j?cPNV`#5b@)bIieMOeIinHLY&o;_FW1{ac^l(6L)!@ss(Mhs) zd2`XI>}d=(2S~KF9x;b@@3Jw#(115IJ3Ml--Ix*UA=-={%IT?!BV;{uPI=UiV%N zzvOD$H^mauk%nwWr^T;Kf$H%UEyOD8AC($0@$Dj178>XJAh6M8!$maH4NXKDi3CSa^S4?|y57 z?*QuFE%}+P;4uPxvfsNnidcA+Sa|lo$O+!aUJZL%haK#7aew^b_57~c5WLS*56meo ziu+>X_07Gh&@$Ry5M{;(lqrYa1+;Jf=z;xd@wR4x*@FK4cJZaaatSa^g|_7P+|3^J zaDn^y_ksib8RWp3Vw}4OzMm3!KTj*30zSE8X1yL`Dtl8c!2Iq3eGe=Fn9Tj7TLa3} z%RZS=COV*uz}<*`3x)eAXV_$)>^W%iW7?7Px4q>elly>h9ba;jJoF+-LtxsjQu^?rQ`# zMYXucOL=&&ns*2CRQCrOPwDv%7PNLhy`${<&T8uolk97po$Y6%zpOc(?O#N8%06c+ z`L3j$!FzJ*7I6=6^}HM{Uag*3M^mGUkiRluCTuJ}3EU1YU{v zZJ;07b0nYF!d7HL;Oq~aQqQ8*)5}d=CU@2=J_TF++gp@M$)O`-M=Ts%ArPv@cpUXFiHJ4w(^M6oA`jq{R zIQZ-jls!?rd2f7H+zX$r1ApY=@wS!0XBk#+cLq3ve>yWFH9mV1oYt24`E0F}7e3?N zf%)*+Ze6B?{#%95tngXc^|i`w?l&d3k9~JeZTsCy9XjQd`0Sa|n@?mmsYwzbLT2AAf2f3fy?{?O;Cn|M9b7|L0o9N+%a|8Hmw838~ z0sM7l82;*K@YmQA@E7Ck;jeqpVHuPSoLgjn<;jn^qXQYPR)}GULI+^)ir#)nJ4cr2 zv#pxi*~T}!+-a^Bb`tA)crLp174F2EvjBcU&#T5AAm)7&)u)?0?*Fz?YcI(+3pj-T z!}aOuL!Y@JeLiZmn@zhN>-1QsBiCA)b0puD9`htQo+W<8PM)uo&~OUz$m2^^dNz=I z%l_>{`==0_)K31cTxHtVD;S%C4nM%S&Y?YlwUP0!05>-BWw(<9NOX_r9Es(Wx#)xW z#2oW{<5D@B<<#yrQ<#%PPe~5AztR1dr{+Px{V_kR9E%33+c zn59~p*IDcxZS;FSbGztIxlcguO>fX_IsIOeYq`6lj^{S!ez5_N{kM`!6tBe4A^%-j z@Vd-r@_zX1ikI~g^X&2Ws6A`6p?uf9BXOTX*tx_s2geD101S3VPaNPAY3Ezr&oI|6 z!@hiu-&fhk{3t`_A)8O)I^MWN>}Y?CUT-l6>74yD;|^q;b+k2kw^fVRXv`%R>`~_R zD01~h`g64izKveOmo&QR{_ILj^r}Lek zJ3AZUodWnk&asO9*8o41@VpCKK;F&Q@SEyW@&Ac_ox`)_DLevx3oI!eAF!UUw4_wX z`HTO<`#t#P#8#(`*`lxNzwVwb^VXC{$%D@SVpF7mf3eSci&r?p`m>X^HZb;FM+7(& z8%@_a4&n$z|A)dkIRsAK#>X#!UXk+mX=9`vVyZ>=|AY6Bifmae`;J^2iQZ++u{fIh znpEyS;QsEVs&_s8r@62#^Az`IF1e?Py9qe&J&qXAYQEXhtcJaBG47+cdfQIJ53bd- za(|%Wj-n5Nxxk=@0uLu@AV*jRQJ#&Nu$epgDx91 zx^Skda_f{P@PxJ0l;5P4W%13T& z4t25(UzpI=1@c!*E-_E*QP5O-Y@stUbRbu8edzrYZ zC(`bUG~_h0Cx&(deErzX!SWf}_XD*16zxm@T6`XDDv?g_&9{W_ZxntsVulxf3Qp!= zXBH!ebFi6I)-R)2lOGg12|S$!JXJPbZ#eAki~)*U?DB&V%J`tN&lqJL(R$gcXm@le&X>DR5`8yu4qfis-Z7APZDNgDsPE%lppG)*)rB`& z;D{5@R!A3;QKP_P37Hu{sQsw%I@K{Qxwa>QsiU(?%|Wt!QUC+d5Yp# zm<|tRJ{WbiEpo_t)qJxhfR`j?T=o9plX591vF1~p&V{8~xwxxs_S1UG*(sYfCTg6Q zvRa)FhEJMD9qjO=%m)WvZHpfoTy|=lR<@@e<+XZW44+g=z4eLA*HI$3{<-Y`tK;|& z&RGYJL+GjN?kM-q=jim+pDpe?cn)2Ih@Sg%5_b|?aypk<)Xp!pIVQ>X`~KKItc_Ln z@MO+h%{fLFzJ!-Plr>9krwx9TxNAFhR~mMglrh&J%ZCK}#^j8$*m=V5W_wF$Pv|G_ zrM>Df?cJ-LolNqLK%YuES1xBC5_LFaeI@Xn&bQZCqc!svI<{3BtgIcRZRV43u-+H~ zs}7IumkO=(BLg0VK11)nWM0v174gqW4C$9w<}fGj*5(D7yZbsPPSW~d?Pl02fo+`& ztnK&%#HLEaKQA^;x?SeLB;TlzdS`0S$-e*mk6^sW_@+VcHO$THEK1ina3gc*#Tvco z*Pypa%cL;)(bk|#jTWdcRInzndAo-Ug;l(uOmYE`tO5jgL_iN z7AUYx4QmUyEUNpKTXkC?w2WyBNSOxg1y{Q2HgP98o&s@V;`SB#OfYy~;QS@Avl16N zM0nWX^IZd#%4pst8}G#amv4f58@%s2ptv_tPx_l<$insD+8!bYg41$vV!|YTUg1X@ zcE5?Q_t-BHICdxc^*|-|w$QJ(U$7o9+j(qo__38UHmNsJ_|6!M3pmAoKCssRZYATB za%WNQ@4WAAEg`Tb{XN%PoX*|x+*>AhJBo~@C>Hk+jmPve94|4()&q<5vm4A8KLZX| zuFxX*E;ETIUhuB$`T6gO{F_C5uPMGL+7r87&Ts^_b+ysfjqmDrMk>I(6_~g17peyY z-j2Ic3Ad%Hz>0e}4L!d%`lk19FJKAB5Nrd?U^(=g?(4=zXiPvt!moLz5={yNXum5ePd ze(oKmio25i8Ck1UbEj%KcLG!~b`?LG*g(Ib-ck6xV5~7eQdh-arO)Gjo~axZo`*C& z#5e=-IM?9A!v-GEAy131;zN%i_sH4!zoKh)S={5~cb@7VYw(NsUPEQ+hniD+?nw_b zr{2i;dYe;!Y~D|9)ccY4oa|G=z$$W;__ou#13Dnj8V%lo20IKIG#WIh51;|bhcp_L z8$2oa7rchjfP2!9ljBfaC94&_Q#-*F$cCY-~_me38@!l;A)5Hjw}Dj zx<)^rDtoG;mzTjCGUwd*ZTHjM`}aZ>d@6HzAU~;wZ6G*NN9cB_Isb4^IyfOu*p<69 z8%ywS?(ayumm2N5!nE5EpfAmK-_!2KPV%h)-buo*$HA<)zZu}4A5A;?^SI#ov5|Jf zPW~6;JC$+OhKwtm{jiljOnboml`-!~(EHwR^xfC7uQkR#ow0ZQ3O^@*vOXl2_X|$& zC$>T$UDg};Q;F>pUabITu`6T`JaF&Zyx&pQusM{ybgo+IV(;;M^pC95IT!VWZQ+^j z?HLz)%Q<5uXR+nm7aPg<-)-?$TiOs3!8d1qn zqC^u*%^5YDm3Uyu-P)>hznHT|yL%%#%4@^lJH^T!Sj?qMEz0DHjJ2F`74UySuD;(} zPM)-S)+DAr?rjh2d~n+^#98+)KC`1AM0>l)11EdY;M9G9G0w)0>f(R)2GPa(eGK)` zry6^x%6dbm*?u}SK9a%RC+1pG5XMuH@yLWsf^H!K|ZRE@P0z1z2i1V0I?ied) z{gWHi)#4LrWe-qn&}#g~$!hXSVi)!urTkaiUlmxs(h{*x>fFnGS#9C$l$@Vp&a}lZ zUR};zMD~)OcD9ovP*LY}rWtc-Gw`;uU)c^^B?__=t#}1?8@XzJ!8@@TGOankHR&TV zm3DLqx;|CA_vH`N7hYa~-jT9G+lK!XTa(xo-X$TAvNzo^I{NnZ4BPF;?z9u96F&Fi zhxl%*ep=|FWX6_r}+K#ag@iqJ{zV)k#MOn(;{B_*_I-GMr_K2LFip%PbO$P?{ z*5vMPd2b`ONAA?gbSSfv@Us@jBv-s|&Sm(akb`|U0$VkH^A$0+3J1^0?1Nwzy)|4> z>cF$xMmtYRY#;qqGskIn^l&fRSM1~MfcLASlKgFcy>!27SN2^-o8`3WjiODSlO%Rf zQ5GRDZpraNF7>Wo16z6jl!{~QrMK;7fBnuVT}Kp0xZ6I!XO^eBFG$9ILT1PZNiGQP zGU85xY2((+%KGB@pF4_{ z62=abd#R4WM>crKmauJL+XENY{u*5*cc!vO-L(#RuOj{`u+Ma!?XlpS{i}K8rA`%p z^#hAvRvekb#=!F{c=p0e2N~BiyIyX?AZ7OTtnH^QUVOj6t>9~IBQ8?p&3=({^&H+g zkxv`3Bl(Ljyng{>ly}p}*Ifio3l|mEN`GG3m-}q(6s$k|BRY(pnVpALM!mUySP zn0(Z#(=kkJG#Qh`wmUMWPA;}OcOC#vYmQ^sqYn7ufeSU5S8;!NK5$8&{||g^j^#D- zx0^UA=yvTdNK|&*cToJBa-Yi(;>=F-l|HIe79WY8SO6c)0ap#^DB*8Sw>t*Ahttn} z!2ity7u4p1Q@QuLTI8I*l(ej2LPyJhXuk~UVbmXh`K5pXSW ze_j2Zb;pEmyi=f^BaJg(T0PEbQcrPY4O@?Y)T~>M4$S0!`wd#&zrpSc@q52-IjzGR zN1P?GZ!3!y)Q-ADshvWMqnzRV8*`7@?xWD6?aZl@^COfAe`Bw82W{*~AI#jLB>j@H zrY@dP+X(NBQi0M=cn1bZaYnEac5n(#a2`W{iMF0n^kgG zruxnWhb`agIFx!48zVTB7?C%JE0rID54rQH8alVY!|JCGYWDm_WFee>K6d9cZ+k}m z?H!{pyWPw;Dfb2*A$C^7MSYBW1mh4GRQO-+UMoYdE#t51KJ*gz0GWDW?)%-d@i`M) znA}q~k8)G_dy=x6%)jOInlfL~(mgwd@_K*CdVh{s4R1$)LF>QDuG!H&&q@sI^wStG zKJKUWxSwxED_!hMe6Cy-<=zPH=eIYj^zko-%1Uq)m|c-_LE%3nLU zV*hk|8@NgXSGw-CY4-Ou;7o8E`QM;>H;as&r0)HeG5yfGw~aP`2;Dm@>HpWgKZNcb zi%kCzx;JB0uex_z8oIZhc+39_bnmv@;(uHBZu_s&y>Aga*_-aY3;gw_d#54`r=WYc z<(@$I=KP}Xx_1I)`=EQbr>ZJJs$<@o$Xq@ff1kJz`njGN|F7I!!6P4hI|_)KITPY z6kfvKXk(5okol51Mf|L-yer}P0Q0lFds~BrInD3^?d4hQe7T=h?CIcrm=s@QNclhU zEU<_z^mE43o>7PWu42Diyy4=-lO0a)xT~DpWn}Aca?V5IOj2Dr-pSbOHZ{3<$hUd> zPXiV`E}{6Xti+@We2F_VBb6?k0&dQZxZk%7>2Prm^7?)$71^9~mobW6k;!=#?9?JB zwuKFwvynBPQl_};@lD$3C-&H){r1sLuPa{dH+rtJFHVVBnO1n)vGi4rv6r)!s%cRF zRSa&osP*r?!Mf#Y5YoY*p3u-RW%;}6Oij{_e^-wDidwQ4iu}lig4|VP4k7=G1h*~wlhv!D|5BLW(S_(#0s`qS9t8y zH~+7v>^-bg&2_&S>;CnvxtwFj!M^#+ZL$Up$cr|4rQ{u2-(Sn;f?>UF`uc%A`CQiD z5D**ju+w|`P3&>+mbfVk@%-5OtfL~~J+Dj86XcaVWF-TR%M(;0dDZTZcbD-idB}*T z+V-k4z`b0{5fqOdTkqiEj?VzuwP_&$;E zQ-}{=Z+@$f*6R|db`IaP@-y}F2Y_j``L0TTS4taan(y}d+fCzLApWNru_Ri1HWOF< zf^fAh%y9C9|+E@A^l4Hiq>Z@^js6tr&bnP zYJ5d|hptz#adlm0=4o3$@V`G#+cNx9-v*kr)_(G=3Vsg50$5_iuTMiDh25J zMs&SDhnk-E?OJVbGT9f!+_j@J{2U~{!rt<x%R{Da-yoL#`DlnF0YlRvG+${F(v-A{MES!zs0Rzq`>Z)a1}rjpm!WQy)rQ%6iwt%e(7i%OBdPubV{-+&t}KFrALcfIJSfVmJIr{(d}=2O+y zYaTyMH)Y$5c`4SI3*+2Y_7$FmmqceBKp($&eQ+K>t=>7YdOf9oP#!<++plAcHsk#{ z>1d}Io55ZjI6udSpPwh$_4&EpoS);}k-((Q(PQkoE~UM*q&*|Ipw{oZM!!nGo_^<1 zZXV@yn8i=QnBar+r7mqs?m_L_%O)P;dg{u2c6DE2{?hinFiU&lquKG3;5>d>|DW3p zeERzrILDc=-*3}kH}m*ub#C+H!n9Ma{fR%1pH?Qqpi^-_oxa8K$K~<682Se0@zd(D z&Je$R5RPV=`1QjWn#WJ88+UxUbjJP~<2K<8u>EwquOY#2_cbK;*nJs{$2LgpWJ`sO zbMU*EGDLwHy1# z=B6#Q`5ZD#o8n9Nu93BX;3$K&LhK}!b`-DRYS$6`gWw~UKND967|R&NRFRo`>M3pJ;3NI0xwI!<$S)!JY)6BI189#G7AbO6;aLxUSb3%#<$SDb93)c;{O$L zACwGqcFA2qSIN4lWzqf>tOadn?D4E%u08*BTSYm)!_QD&AP3lNJNDs0>aAFEuyKyc zo*(bbwbTlqv6fQReVZ-hGGQ$g&zdT|@b+VQtf}r`P4$Mcrh1b#)gjhYTjQ^+IK-Oj zuEG7A6R5LJO(=4}TW_+a0=L?lDtA@d*j(0ASFoOXj5{g>hYGk3*LDu;sV}wlRR8A8 z!L|xJu-ICaefONHc*Q3jxW6xWW3P_&a4~yb1)akO8*#+izN$YafJhB;T8h zRLwryO&^j6^T@SP?8$QmVX=)|f)>s{`|rqAiW{;GyJK63-LcJL*c~m{y^@Q&rN7u7 z5!$)Q^}A26mDv9EYxKD_g?KW-ar@{TZ^yV?@23||^}gO3@7s(`A@;^-cn`j)xB}jr z-tbyeR>>ZXr>F64j2hpRRnyee1V2y5_Aq&M6y!BbghG3!|HMf;0co5w@LvFH%tr#CIJD=me0YW#E;{FJauU!xCT@ABM% z_3s^y|53%cl&Km#pc$V@9ml_m@R!dX!2ahsmIRHT@~R#Er0#J1C5Q2sBv3bjx}V2i zRgpklS+lHCpYSxPZjZ@(SHXKvz%Ni!&9=ezihDOG-g|~9 zb;4tkyY$N|FC#|wQqBbH=SJ*>pG<03IwvZs^IHX&v|R1Th3vt`;rF=rdc}JiJS%N{ z4!_A>!*B3!39cHCbnmDle&j%g;!XoM8T<|Vxmvl6a^f!%o_^)K>PGbU3vbSIYjxG0OekeS8d3Z@OU`_MPyc9sRJ!@(sjN4Yp=o!d8P6j3vi3mkC4Rzdt~c`G)li#y5OFR4 z8*J+C-4Vpe3_wQ?#HSSDR*8+4vmXC!VSE|HnOvkSR5%~kQdXbc%DshJTnYCX%DHdZ z-)kcVq*2apmDt?)^|~aV#5CE5ByIqC`#g(#IoqC(_Et$8i*ulR6TWNnE)lUkI8V22 zmD=Ck@w-^>X!gfCZY9qTWsB5-`W-GW#d|wGP~DGF?v0#&?v9uGc|Uupzqk8$Hg5&* zT#JYoDO-`<_IwO?G!MzDx=ijBx428iM-B|cVLbiq;#tV|kML>fcbw{bYpNTapv5(u zy;`mO#}oRxH1NEzHm)4IKCUk3zjE0Bwz2lKm*o6y6gh|v#wvBOY0f(4`=S=+#SZog zZ(}d~6ZY@tsIf&N+bc`r@X?XOM~%_^k)s}Rt)8yN`Qqksk2K|1e%sL~_JRXFC^_$> z%{ba@x5kpkCbp=AInz8hY@QkJCN2QK+7rv1DehKibzmH4kC~Gk;4^Be9=Fl-sKmfa z{6?a$;Op-3htMfmr#n|m46luR(R{-Xthhf!cU04sEiK77d#4&_@5mQG%mI7iPo%k$ zh+|OPBQ+dZSi9S~-@)R}AK~;)ACu#qNgM}$otpSbx!%WfoZeO3Ptdy2>TMrMA7u~2 zH;-h0`iE$5_a?>L(XDz9?Y3w!DQ)9oxGz$_2R;RQeYHy_5J~>*GY{ z-q&BRmkB-helY&{ct8za<9z%9t?eYATVRs;AWZ*%>qGw~A^pE*w7ZygM?tH=@e-$@ z#W^Q|*SEDjh9qCd1Y-2GTrx?%hf4T=9=V~hX9`M#&mpo0p~Yd#_qUBcZL81|w1Z91 z6R$1&F7etkZZh!FKR8bEncnbXl>4;-{|o~!=9wJncPM-sgB)r2Istr}`v6}?2z*P7 ze$S%czdx>4{vZAVaVPqs3wprau~6Bzc3@|4pJtNew5TH12|H(D zi~Le2hu?w?GbYQsG<0UcBbQIM#VbD=E_bH^Yr#`Fx-5Efdg2jNpijXA#``6_XB{Pe ztps^LmiNfA>pcJa7~T(({_%C=f0@ntXkjsZiC?4Ootb)n^xp>jLG2nbbYI8O?j1&d z3;b|=$$M96ta~Rhas|VT@9F;UpGdtBSr$0|Detiht3}7Ko-^YhTjc!^<2~noRPs{f zQvN-8f2Q$%l)wDjymytV?lsL4w;tjCGLmwS8Rb^`%Qb1`BHU{P&IhnZMNYZ%&P9$k zv+oA!ldHd|$-6SH^seG3J*Tv+UnW zJ@eO<9^pQVaxzv8$4mThFYjfXB{Cjj|5@WyT8#E&TryXTMt8pVSg;PTDKoWuV@lv7 z(FZoiWtt4s3-7~A4&DpDmk3WCBsP%!OcNhc#>B^5Y$;c1wEGJ1;h^6jJeKDB<>F`4 zgXIGL*2Xr$&u>@B*iwx5L;U zU2wf@GB6=GG7dTCsmAO2O5*Ju%J6qgn1mNKexhu-QPxIT*-Mx9chLULwBOr4zMKne zBkowU@xsh&|3Fuh)39q4bHP{eW_7>3&KJ{_eO~N_L99I|l4nl#M~cydwz8#O>3a}% zlk_j=xO!{%og4Ia-{qXZiQ274mcq5$h|bAmo^YW{%s9Ky{hL3>3aRTtxr3@F#a zSb8fbOaw;GC{`+LEe?1M?$GKNmtG~1W49wk0e#wzX07y{eBSsO#LFOf0qVhqR4 ztHf1E9nB6k=AvtlVA?2_kjw?6zhG0D9(jx4 zx#?$-&z8Lc8@@(|8q=(vL5@XaV~3O-YqWiyU$<|S_ZJxNFZRFp%KJ-=_tX9FH_ChF zoi^d~>v?bRg*n!>F~Phh;}O1g!oMP~w&GyEmiborT47&nIJ7$MyTQ-${Xu9XI`7z> zzbEENCFaS3&o4MfSa(_czOOi^AUT6N@fVA4S#mluhi_AgC-$7fk$5YcL(idDC3Y~e zB|*TWTEX+?ui;huHpgZ9tV2|kDaYl%DErT*7({kko+ymgl`UBmL>MjudY@qzu}vl zsnFV<7oF9WtJJm4SG7JRKKY(|DO1!htL32x;eTBsU## z-|hGp#E;{Ucz-RonfO>rv)hPiZzD%#3vEmOAj@3t`+yI{*Q34TJ%?4EZp*{B%h+1L zOG{$?Ma33;MWdOM@t3x+#z=+#!83a368th#p+O$B$b}|3#7VuXouOk6eM0$Dvbzl5 z(tJmG;To;UkM?AHhoA1B=ER zvQKj&-VhuJE=mG;<9v8SJ8#yY^M&wPEA14*AFa$Ep*%7-Ntu02aQ;1f5^(}P>E9cl z?6&ss$-#TzlUTLx>;7t8v9&3?Wi)hK3_mRA?*^S$TG(T28Exb-yur^a&i>(frOHCt z#NvypW8fX)yTdB>bGv-Q zH{`(1N}YgyjJ~9%WqCAy+DrMh)x1=eOzu<5zN+QwzV#a|`!+By|LUIZr>m{kX}J{@ z_A%;#rONtL_MU;_*AK)l&AC?2`VBBRZ<4 z?MlkrEqweceyu-ew^}1>o@DNQ@qK+vEsK>dm3h31^TpSU)%^1M{uTQc!ULkKq`zZ3 zWDGj&ZFke=TK1Z(m(;YDMc1?ZHsc4EW2@tAX%~yj=*tf4lgn@bFFW@{REH4e<8$$l6cviAHO2eww@uN(q)X zCDpwFe9k4t^J7DXg5+&vf5TUpAh|`m1P-Cy>6DeZSMZsS-zgBDVT_^7_-?~K(f0Ph zO+x@~c7PjMlSvyb(0mf`%b6PS|CsHTGZzH5YvbGPv@2&Sq>RY7Rv(x?AqR)xp-YqV zMBnd_@p+V;%U^ey$oe4gF&MlI0Y9gKr=iF?@}Bqa%lhQy;br}fi14zW(??mi!&|-0 z2}!fU&k5CK)_wXps1)C!yOqj66phR><-FQDtJ7O{b*Cxk3c2O}T{)k}ywfA+AE?N= z^S3Vv3;}Y!sHk^2KYhF|=T*^@H94<}zM$qMYiegTZE13zr_1>z_aoH^Rp^z|Ysi*FQndufw;e%ehO=C+Tv&*O*uLE(?(JPgm-5 zC33#koL8$yo~OzAE8XL`Gdi(T;1D^_psdK4;8Wy05FU~96~^~)axQKBVSBKgUkv<# zaxU$PoCmfma(*}MhRV6L_b=Jc0T10G@8Gfy`7WUBt^A$P=G&dHubcglUhKXn`e^rU z6T9!CN6;(4*udVGY4_dDenn=8-PZu0exKc!0AKYcZ6fdIW{LLAsY=8WiQuz*Rfp>E4 z?nYU2voDA*QQ~Z7dFMGNvloB*^=7ZzXSh@8rEL`|BCO@NKzh|J~L!gC(aR-TD6TUQZzgJ;@-3sh&oJBQ#k?WarMdyg^JB?hGBdD+2 zB=I#Z7Xrt9#n?Q+u{@SO#5ReePvBT?=NUN4Vt57)?66iVaFp3fd5tS7 zs;1>W&g)-5Tgzwg96`C6HFGVI)SJe$h3Dx!TY0{WXNBi{p4A%BS%E&8V|V^ZTN_ki z+pXZv*)d^gQU68v>sgr(1IuxaUGsfDM*FfCf9%fw z@L~eI=--nU*plxTn=&%QaH2h-p|&b!a5h6nQv9;_A~WL+ue7G$r*tF15X z9U1&z8NeT{tU+nJ^L^SC*v$R%BiCNV+;OG0Z~o73@ZDptiGM$x__(>2Txf2AUODi9 z3caTC9KmxQ&(<2(BVyY|)y!Q9-&91_xJJQGtS9DL;1}Xt>~XowgG!CGC$?V+^G%v7 z)x8@#SA!dQi9v3B!-xk!hVoelOWF2uD?AbdDRN^w%f8kXb!m+f8CB!DfLM;n#TU7v zrt%yCJeSm{D*Z;~@@(Nbk7wYDa`LS3oWrwP6NrZ#{I-IJaO3~wH`w0rT}J`Ft}ys_ zeAzww(}y_=2j2+|Y+rVduY$+gc<1=a&v(!9?gQS{AM?Naz~n{vErIb%KQh0@b?Nh1 zG`#U5YsMY?^_H7Va_+1lk4G{wgR;lV+3)Q~Xrn$L|HL42NgU;!_@iv>yAM(att2PV zKFNXAN$!cjvPUhpeNz5@a!=T-M(zoTUyOfXbr-oOGRuD2S&a?iu+Hh(QyK#=XIgLS z%w#XfQFc?$`!jhjdp`x-^K4fWiCHAS261C)XQ-NQDO_H(!J6D0pD#`PH{DjrQ5vkT z19LpL5`Uo-gG1sX#(W7sq5JZQUz3>Dc!^&d<@DM)U*;fwP2$%25eI20yhDpyyOY?n zLq;6r=frzS?Acx|_AF*)!hnS40hT$n>nsC`s^FE+(c%3q{k6EYS6IU+%zF~QRzw@@ zTWv3*ei89v&G9oTiio2$Hf!euh5EO1w#P>w_QKJVR;kX_G5GJMr7;hUk{H{#=IO|icNDQ5 z17e$#)Zw*>CBxx!|9${C)z*O%}7wBIxV&P)7@#$kLA{H*L zuxQfzeE(j$u;;ANB|~v9{o)6wODA;c_#dK6Ep(|3Lzh`6NtaIOQrj20yg&a3q)V*g z)#!r$84O*7{vtEB5SiIS%%>?cZ&*%BW_}Rb#D|fYcaWKIu$VHV7&0U43Xzw%Q<0a; zr(hR`mlc`IL{28pLqC6yj2z&*%>S|;Dv+4wK%WpY5%sBVmsX(XFTigg=cK~Kao>_*{Fbm3rA4@PuOkZ-T^{?jtGQ*>a>1vy^gu$xU8Y$+?w7X4`I z#yw(}pNURP7M(bZxbFIFCFeKUE!Ku?MO~LIxwBYz6B`rkr#9p8L|2;hml)qe*apvh z4-fp6?_$r1Z7%a?JvN!SPkv*BZhyr>|Il**a&H&+S&wQW9&h^kXkrrWP1A^vi+RYt z*7nezwMm?hu;F)Xv*|SN$DClxbb419G><1IlFW6FYjpp<810yqFW3h(pDVVbqR(}K zG1~8+V6N*3nd{_Sz-OGZmKbrF>jW;D>%N5U;b1Z8F1pl|*_p`f9n5b{%x^;1KpNUJ z{qvhWQ}rs;v)N;t)dta{{&NCiH(0O<1osk0Ei!A@=C(vFXU*@vhi)x=@3vZzM;rFJ z9s4{6oe;}f(L$Rxt{^7P;O|ba*lj=KOr-1)KeR~BcS384uheLkH&}emb7g&vtP1ZG z?MC0S$I`ZuXG>DW&-SAO$crYvWmlA1BYtMrK>YCJbt+iW&)YUd*7CdfzMJn7r|iV0 zjjAbF66q~xOa)7#yi3!gy-Szayah|_{KoJLoQr1w^K@XJhF^%gudG&WzTbS0S}FOH z1TV7Q`TZ{4=3F59ColXN3Mo6>#(sc$U!cMSbKW3`7=2y~{!`0`NdkU7sdzVg7x|Zhh_fvdav#`^szjPMQSMc`>-sR>@ zCcbOJ3gks%+uJe}x04wEq2Q^&kRg}Ikg<z{PIvd2&`3g3Yd z-z+#4+p3(l<*dyD)==lO_E^UqfSgn5%EHEze%Fx`p`8C!_*wR|ZK}$hDz=V%)>ikh zW(jwuKwI}w{s25u2CX94S6z4u{vu*p0^6{&zLGtEv)nSu-by*Sk0NjlDszc;PN@Es zPKCIYf=;DULyLjj%_ zOZ7GNXCeoA$KMfPga4CW_umun_nH2C0@mO!>Fu6?t1pp#@!)#`s(2T;CNS>_2-S%) z{=|2k-Xrjg#Lb92$z5tYc3HL9j}p!Y2pu!wO*7xR&_Iv%@au-}EV4%TKZ~yDHNNES zTw>|NF=0 zw^cgl{Hgj?ZG6G+dK;q(9W#}jx27n{VcU*%o;KPSJH-L7YkP6GHqIRTx9!DCv*ID7mnMvj!_TAd`{1pGhp&5Q|p3Hg%#UZLNw*qfL5tb$(pCHG`d zzdtwHOry=c%Yywvq8mjPstx!}y>tt8!o|JJk@x6Lq2u?(V9b`XVPY_@koRF?FwjL| zV=!{%{Rv_)F3|YJuN%V2d(vxuS#O=8R_+J>f~drbWcW~%_g$&(r@(ROy&%_4HROGs z|6Y)1co$CI#h$6go|&dNCy&CfajvQi^hGP%t>{oYd}P`}qp^*yz!o~|9H)0Hc1{a+ zP6B1p7R|4{Syh`Jez4!#We>i=wT6*R53}xHAE)+Pro^q>Pr&3dHn7ho`Gx8qWz&(ZcD)M#??_{6DtYcFX znr+OxB1bYt`F@F_u@2vHXRxe9Ty$kGrhmW6FGNk8b{$v8+>~i(It- z7sm2WB`Rnvmok=18Owa)Wo}U-E9A_az?DEQpELuuH0p}YAn(U23C$xGjja{@o8Rjh zlhA0yqPexVs1Z%if&XX0|FcR|({tdz0sQZ$zR)+npRzp*c^Soi`^tD_yZAz4plvfY zQxP`P9khG)0BFy-lC*(3ZDmc9yXf}X4^&Ij&9ptAw&yDmO*hka32o0;5>}=!n6KgE zX7I7hz{kVj<5}==OWY+D&r;?LTV(UI;A4&wSyYc+dl-B?qeKMZV=DNV3O>YUEmN%( z;o3H7D{$4Yrg(+3x2@n{GPIHBG0;igRmnLj=yp2qb2S*}X)w+Q#`y+}Hv{7`V4NB^ z6+TK>Il>mvybKuMS0gl<-3*LniZuwvJYf8QwbdbDJiz&Cq1h_>HDO_%nSD0+y#kzV zMNazz=Q-ds!P2j3A^2673|O16WZzngMQzFfhZ?TIqu~1;+SWW2Y*P89EhnK~c zm+{*8X7VIwceooj68ka=J2M)4lXJJmSs_`kimj6BLT_+q-e7BLv*aZn|IO0<8&>Ck zy@z=qh4#ePBsnC-_IV4N<4NHC8l8F?aoQ=&0iv^2eiL}EB6lx#v@eT0(ObyvDYAl( z8kmY_wPz?^_CefPIqc=kVO5P0W}HnD9pA=2is*Rk@17XdPqcW*MBiHGgdZ9gj|br5UHS>c z#p^u(P`LQ!6PbtqeO#~*)8@?U1Q)v_+1D5V4hMqEL~xn}Zj<4S6ym8O-Lf9jcmvuE z`q1gkv<~X{dR)0D|NZ;-rwkau`cc zS+DAMKw5lGa94@W5*x3C-|;+4&Q~Y$R`^}+e%5Q`K1=Z(1aG@&GnO_ThU^tnKAr!$ zzv=KiNwWlmzQw&A7>k@BA3=|h~KrjIC4r&X?*GCH*&a77JGe)8mqyx zH0r^B$sF}5WkdwS8!}h<4|)))w5l()Bfj+(jfjsD|{l+UP5^{Hl=hL@`BYMz&bsR; zrw|h%^WYWyUyKeEnkFksSBj08K-NuxyuhTWuU`!KbE#7z=X&dhum+NQ3WC?Z@Zv0$ z_B)FM;wS?6L{%9 zWQ@uDUvG}_I>jsV6W=Zh?e{&iUjv^u>S^Pw8fuKwuLsQiE^Vyzvomz8MjH|#k;dr8N1{jpu9Quclh6=ZwtOY zB|@nQ?eEti{WVcX8+Y-rpmB$exww}x|K5PN&=_-Qf5jpFZ86Hy(DMjN8regpFna~t$=q&w+`3xXvM%E z?OPS^K`Zw|pik3grn=|zJHH^+{Rrh=bvfL(5=;JZeI~fj^I+W!-3DA1oR=xIzp9h% zr^TV=>k8M`kxN7RDhs~*nA~Mrj7BT-o!zP57pje=evrhyKe5b}rjMm1&242}_7U^G zj75APs~CsGMM+LD$Tj2Mb`RmS$!X8b;u1dnbO2*Dg!QM*69vuYFMiJky`&FytEJwiIE7HHs zso%XqjKS>nG1#iCiK{IN_n=tz$=rTFXD;RcVl6HvqNap+7m3$WBwnkmEW4b!t?g=J zy~~zm6SLacb~bzBA1ZeeCs!l64ZHB~*Rk*V^*HVyU`^v%?!P}|sKjqz&piqM9+SI5 z0=`{ujo2sOA0%F*+>F=2W|F%?@~yV!q>?$EPjOeqitmCY9Mc7C`t>vEb zx>R*e=LA*R-k{l&DVp8)lKdY{YzX823LPlxE3u(}N&os?0IAr{DZXvsws!wSZU5nw zjg!2ulnw5D<*yUC55wYa&m%9zdd0l~oV0Qm!kyr!jdKGsKgl>`JXP?0TamGzy3@ZO z_DlM2n`6A6?|;9V_vOT|3{{7&oXl_C(e52B%k$U=!OkAZ+9jQJ+|{h(reh;7&*44( z{^hy+VvjGE{Su|-+r`&G^J}5~HSoZX;f2}Q*jJPPCrY!iWt^qRUT*OSWGz`+CyXVx zhJ*akqdyjVSk7#4Z(NLize+Ve^@reT@dpe4%bc+f-W1zZdj=2ZYx7{DFA;q0qAgQL z-pqJReSf{+9J$B@H%h$i3V7YGZRI!$$Ze=^U-BAd`w_OHZet>MBo1zZeb z+=;%9C2HkU8Xw5|DbaT}W0JVjG~UVDy_vHlFS5q~e!|+{B1_xpL!V0%eWlbJDq{!N z&B&<0r~sqo68uVhd+8n}N$dtQJ`KJM9YI9;sy1OMqao^5lMg3svN!i!HxJ%SYL`hk~&#h z8y2r4!s$H#A1}=j-X&MoZarowFrRC1yHmV+yCbyoSuN78+@+PL>}%&PEwjBMtv&It z=J^KnO|Z1Hg~^`m^R6AW+z4^^NdDM`v*(4oDS?y%;rud z_e;d^apv9KZAsa*0T~cq%$MAeBzvDLcy<6!Id{LM(C=7qB6?74_z}YYz`mV$!|Lw> z_JYr39$+sxRY|ST;?~(SmN_L&(*a77v?ukoy-?bfxmn~ZRmO;Zxp#@^M!#Ke?lZ}H zxElSU*!=TG1o)OUcCY+(mD8IPA^RB2Pm#(CRrr1cpDDC~Vt031Q<}SgPjtNaa0c<* zjsM1l|D=GuBYWY!lcuPag=zR6yyOK~Ks)65TRFIJ{3Q7fo+}0ulPCWx>Yxg_^GMtG z1?L6*lrG7oAbs2ED~tW|QS7Bqzgg~JwVk`dQ>g7T+naU&Wcb*~e4BIi9QeZu?9-0r z?Tc*Vy)n!nfx-t#ww`B?ldF^qLIf3fT{jN)$!=Rm8VtDXH9 z*@KCRL!a`OX;rpQw>`AluEbP~;!nNQ2{u%t)$e9T*$@@vXpTzq~yq_er3MU6i!2NclvR&p?iFai#;eI%| zZz;vs5)$t!v7qli3FDMYAXePl+2wqx6t+l5FXlnxYg2gIcN&?yjqSjH=iaiq0*32lCTBy>? zAwacJ)FM=Sw6!GA%Y=A=2$~?6@B6c#=Se2RfS&Vf`+A+%?~i%SGtcbH+H0-7_F8MN zz4k5Su0+Q6o;Q`B)QZ~N%OHM?F^{^_LLGtBx)u4%E$go-gNYG}<`SowOB`zMn1(^Q zcUXs*)5f>fooDUaP5zyH_#wZlsPnIv+mZKKALaMVhc+L<+vVWPTg!H_);EYpKN})+ z)LLd$&po7Eh+1D?2IuXpxla<~dMEN?-oT59aSbk0jNLe5u?G0lw$8QFwvy}Y#R}e& zcYfb);$HUnPmb2+r9GTR%;*628WfuqG4)RFnIi+3BQp>C=QrtWCU! zzTDCN^>;hawO)Mkox8T*`O=z>aTmSP@%CMBcMO2Or(AUR_BFg)^Y%*}r~KfZjtb>9 z#jYr@FL?Uvv~ky|Pjy^;=Sv-H?tHi76#fhFAvb&dbuYj4;yiMzD<1D+=vhu~^yqJ1 zdhto}o2K)BH8E~gkG-tuJ;%5siLfMss1bTvL8+UXbs9}2t_Jc2Q?3s z_e3&M_kh1@bXdo}tuu5d2S}^#0w?Z-yTB9Y|ICBQWrFl$wcA9y|IJxq&Bhci+Cn;4eUsQ2%PVaYlQo|5@VWjkcDhGpBC z?Lp>jC2MK`8tg|#N_TjGoVPk}FM8|-uic!psNpV8i{IXU(9dsiY(3RgUL3Fd`;=9= z1wPBf`X%~KgFBhqD*fuV*hTz`8Yp@tM6Z${yU& z``HZh;t^KX@`L^+OoagRu-TtK6+aj*X z#4x&hTN{ZTON^(D&-)!5>O%{nv^U5NW;UmY>T>Oyb$Sz!FiH!`OV&x+`d)3d;__ObKb9Kzl`t}*nBG04g} z(8#eDvyAWM3;eDj4ve^}4IeFWd<-#cvL7`?Js?3fCB ztjUu<7aO%=)=52i@~9%ypF#k;_0c%dDwfO2KZVV$hE@_-0{rww!qIu zg@de_k+H6w+IoU(FUwA~&7IFNa-kSI)o+D2og=%q*G^S@te0CnfF2qw2u0eS@N>pS z`GUth8{-Ucp!Qt)J13&MV2!;!-m$kEccAz9ta}Q=#n|m`|FcdE%btrpZbcfmSg%&= z8Cte2a&2#D&|qL7es`*~=J$bt{AGzSFivv2YRt)DSn9L(#=)>H@F`$elnBF;BmCLL z5%<0%8#~1FO_y@^JoihZ!(YL#dV9-fdwkQI8D|>29K@F@AGRqMa?4frmfJk>LKa+n)J~-t|45V25+R%CBL(8@TdrFNyJ1qJe{*zBNhTV*A{%RX`B(~wQ zZlCtm>sGu#&hr9v>NN8F%by~DO!uGc$Sj?a#MfkE#iv@~dFXKFopo*)-%9u#eg`;z z+R)G9y#9!-jhESxpBvv&-#O{&iy`F#2o;29({7nHE?mk!*!efx%gy^{_kHEKu-%V4 zuy`-|d$h4@?UKJtdS=>VrAO$F1CYVeAJ?Ei-i5As3i{(a=#SC|Pep(HDer!Y{&=d< z9|yhoE@vISiSGCVbjM549iJNTZpZJrtD%6;6X<_mfB9vT|NbeRb0e>P3;RDWqHju{ zeCIbWzj!ibus>d`CSQJg?eFiKhYpEeT7NP1uv5$(6V=qKsC|)q{7zdY=RW0?kN*ib zzy3Ge_LXyAcTp7bJ6Sm|U;o_SY`sr8`U`4bezAzLwV`ViocPiT+2L2Z`Pg6Z^0Bjr z(2PxAHMsYjPBZs3c;|Em+4ifa!J|62vDcj2_!jYEIj7EN4IA&~*3W@gEaJtiL7{q! zbBvotT3d6qpJZj3-1<5ATZURgO}v=SCaP`;e5Shy$Gds?hYnI+{_aRaWp&xL|2d^HjeXqON2XMh_bteI#43DFJKfywwd9aJd`oh^HRrhO-rW4j&Lira z$jjhX=YQ09%hZ}Fty61FpDj}tPkCvO)pZ@V=Q8)qq->CB*k|Mh>f&sE_yy)oU}XAM zv*%j+BLidGRDWldJ?2@v^gqd^0PH#|v}Ln7E7S*e^rmv?R_C+2ey=qWJ?z0R+sp6Z zxe1$Lm%sO3fDzn7Q|p#VHs0^fw2eJ}E9u%qD z-6P{s-Y14g@fw>XXY-uCJh^&2eR|KPEo5V2Y)2ONYs#1STTebU6!yg07oiU1eQua!p7YUd(`fhP5qEEY;;{`K^9r_h)cZm{bmyRdi`S{rN|F&#<{m&ljn4a0xF@51ar~i!)Y=7eWPjs}uv87}BftIB86ZhNW z_xGz=k4|27))D*rX05;IhO@0)rzo$S^- zr$>j<{{DC~=3d<>+5Wz%zwXHT+TYJjsGpBK2w3@@;+FvY7O)G9-Ip{vhBe;t1`#AFlqN~}@4<5eF?B^@av&@=yP2=GwEMANPGkjvA>@n=oA<9wNTvTOkE2_E3&iv7IJM&k3693C&{dtY|`}r8VEhR*3 zS0H*Gy1U8_;D3F8lQ#I?VE8a6RE%x7LHTn^tYCXfM%^fvKMSNw>+D3?kZA{7PFiW+ z=U|6efw9Nux9s6N;pF|?accwD|r9$>vJi~{V~S}^Fv*G7F9mozNGSL>4)6?(OyPh z709WIpPdqBE^G)E*pUc4#vNT{>afcAd0 z<_$d6^qqF19cg2JC-$92Txxv1a$q*~{^r1}def4?Jkt)g?*d=b8Ap17T|m1V`iyB@ zsOxRuc?W!4hhKN4J<#C8v|a7MlrEU4p9s?cX#1qknlu1?|0mp1M~Z{KJSYWJa6 znKtfl+87}Cl8np10?zG$)^9vKzw{sAd~#^RJK%XIXK+htYY=?;E@w*zW4pggUwi3` zd(G;1+C$pQJRE-gG=syz^i`0ouT*Q&z#OOkQ`GGS&PM2C;4F6P4y0}}+Do?|vaYmv z=;@)Zp_{Bu`-ITp@zA|wkTrkb@Rg(T2Bi!dy0)~~stRow;0ri1J?xt&=)UWpF0`)a z?zYbl*}L($Ztik>k2&&+UHSP}ZX0p=&cpWnclo^${(kq-yGOnE=nqD%q5OMU+`+Z? zDbBA9nfDIQYhL4H4=TLNZxwm`I1XHSym*RL@Q85f;lkyqZ3QnoG!rhWb!T|OJlF_N zEhTr;TyT;$@kpLpDqeAE8qaG)ahow(Bi!#X?l<#Q^NTrazK5V=Th^RMV*jw8E}@Jw(Ry7kQwA$ ziU^lEsatwt-jOAfw&1HoPIj69?B4%sSi5`!$xo7Mo4XWy${gO@Q(x;3`}O^|i`%X{&Q(LI{d!atdO zq;2+x_Z|L9-1Er5w<29lFqecv^EHPmCN5i1V&%*)7_V>N{7m8pgr|U=Va}I_{hxSbnH3lJBkZaoRaol$I{*jd>+GA)&7X|l}3DJ5p)XeYpTBP%v66q z@KJDUzsn0!N1YL*y&!uE1)0{F!BneQ@hO5&u`30)4!FD^IJ&1y8s!Sd^*b|-a;ALH zDptPRf?JQfykOv%zU9svV9NE>p{%Jh&?>I;Wz_3D8gmexuy0&@BHkRI9Weyp&%sUB z)|u!Jb--80*epA%UOLWA=s1nETbGu+UD>4$zI1LaqX%DKaPf6NxOyA;Bfk9Lsuc85 z?!I>8x`ON@g7_tP5cILs1H0o&2y~;|dx53HDwq39LSKIL~x6~6G_XFV6Ig5LOnGfd?FB^M26nTF*XV~pP`+fAi|IyNj#*l4iwM)NU3T|@1O)j+8c+ZT( zPrSa}SJ&1(#NsI*I0{5R2G(kH@Pb)-@8dnO*{sb6c<=2|Kd77ywEsKq@sk|c4?grQ zTG{BEI#0v6)}>i?eHHpn;GNWPaNFX`@Uv%fHt4uJv-pf0Sa{co*5@~p8+I`~o^7SI zf!%De#n;Z>i3F^V*8BMg+M7PVr%z=qt#4 zEEpN;lKqy$Jt&pH<;5(Uez&>(hPdfWxf$rUdyAjAryy}GY1roJ*ysb`$AR$UAo$Tw zULEZz;8W0?r+wz?|&VXrQ+!`mj=%PX_2qL#7vGk{lZ<{(2PzqNN$naBHqA?-nJ%RT+v@M7#A zm1(f8Now~;%)vbD1+9&TZ@C0`F9z<5fPXsUs30DPI}Eq@!yzmC^=ACq+M`$fk<3x@ zh3-3u-+c=0#Q&%D_9OLwjjecD^4ksy$G+V!(hdCbOT@$9pFQ!WkNOvN{hoL+^po!0 zR&i{qiRVQwve##DVRdy<|B%M-=MGH9uROf1jDM->cl9qKZa(}W>-Yzpja^U7l-4la z=_h}U)|@*06~d3+3;q^z---^>c}ml*8PYep$8tyNXsav0zEQyX_4VpU_%?FAH65Dt zi>S};t(;Fzw{c1MYw)?R(xUE7yl^3#rj>t5=JT=b6)%bs=+T1gMw7HA#h z`gi)d{vGfEj2LxkJtc-C>zI6|`JHTj6@J_7<88C>EzV&~bF6`m&#+-Yco+QA%AQ^aWpX&r zp!HV$=d#wTfA0)KYlaIGx`1Gv08CnscQbbb_yhBXI`dt;9`9@1xF!2p>4b~nqg7k7 zC#&v>;8^-$9<<3vKMbHBa;NU(0D50O`eEaa?8#-66D_18qVrI0z~upSM6aCDCD9Qh zlr?pBXHT9@9mTZgqbCN?6Z@1!PgL2SdXzWy-p`)An0l)+tXD10XlU;N_>Et%M+XlL z2|tHF>sWC+!doyW6p@}3Vysm@%jn@%_{?r#AL#(}T=3=6i1J9qIDajl!_B+OBFg`; zfOj{(Sr&QCH+*X?&t|@nLu#YWX=GV5h_7z1Ay%^vU27v}sdtg1w+&fRP2FX@BQCgJ zZO!C7d*wiUO~XTz3$v`rHxIK+o~#Odt;^6?@rmtgMW-3&AJyK1E~tK5eOdL&SGAMA z8Z&g3YIwc+vFwrcwmlr3E-U(9(0CzxWP-iK9@%aIyUHkrd5>>Iz2cG!>`_}y|M)x9 ze_E3M3*!2h|4nh*v$0{zS~SO;^Y>riZtO0e1!D^O>uA24(9s^*+x=h>bdtX{m+$-d zokI+h+F0bb_idp}z#6qR+qUX+K&qu{^U&{24nei!}w&G+UK+uY_$t8ZUyh3~Aj!tbCr zR?_C|xU=gd5;kpaBuAL^S9NL8Z zQl5?YJR>R_8!I2ZV{5D_!zKb(vTeRpF5iTc*Tc)@(LGf)yjGadw8^w|Zv=-^T&h$2vgm+Q*ts?Sff7svTx$pT8UzlY1 z>-^X9LH|hyeAZon+Fj&61#aZ^s-!$jLh>scY~q~%+Y4EC@* z+c%T@1dnC=E`SF;oj%duAbbDNH*Na%0fQe{`r7`|Z=YTKgY92vd#Yn{(eFDtpMSq& z7&=xn{5t%$5f48^yjwAM(3}Na;P&2$TQ+uH!TfI=71|K<2d2k9+q_~GG8y^TzBm^h z99l-HJ7Z*^e#Lj&noHR4&4tGKewE2;Z{WGkcS~o`UUY9Zdq_D2p@Xxaq4ZD9+do|c z4w<`uXU<9Q{v!K9-havZTmsGCg|3I(`sbMXW1?PvtN7p8B-jQ4KJ@vsvEn&=?!!Y5 zU3S&CAHQC{7ko(@vhCsXV(7+Ivwg+n7cn@fI+yheU0ZdlD%`bovt3;Me5vkPD5`qi zE|wf)-Cb9QjL9gqx`qM|JZbISNfbgGEQ*c*LC+VQn_RGTl8(kcw|Oz z0e@cmInZUj%1~H?>}_cq%7!S z=H8+1_4o6vIkD6Hml53z{5ODKjpaF>?}xYd&{p|fq0W5fYPOxxF1gx&VDcc-jrF3zfb;*^LVb~c`47F{a(D7=OsLI{(JH5Jm0}HXTTTN zOv#7uq_fXjOk5akdpUBw+_)axQ{BEk`%mgiW51WNAMG4s9%t^ocvA7&LiC1W|FE#^ zvgR`K58UPK6TixbZ@>E!H3QLLic+6BIW4K&F4CGOVNA(h+g;K9Wy&$ z=Z-9`H^>0m8ewexO~_Pa!@AALRK5F!$&WTXYWy~=(@*oe6+Za@xm$%UK<>F2%ZX)G z{JrkWd*S2C9r6=MzLc=e$d@VDOYU;+niTKq?6mayqn&?MPKQp97c033kFt_ybK%Jx z*Je$G^Uee~>&)+rXc4#;&)#UpI4t@cv1ppF<9XlmP4S@e9QQqQCp)lZk%_rQ|Hurt zOyz#9GiV3hq=}evQi<>>XdkDDu%>SA))?IYp8ecJdH6MS$4AuLr>!)7`J|*+L zx&OuaedU1q`j%Vfa=z2P;NWEu{aDQTAD(>V9x1UN+~tMg;mL@n;ww4v!Z-oV+|cD3=31JRj{5`f!RPJGAB`>b0Wzr{&Sd zO+G<#c)9s_)>{+$jED7x zW5D{2gnDybSVty-RXGeiIgkvupBAs5Pv|oq){0}mIyIqQi3{ty%t7zmO*L{o-k&FV z!aQ4>YxUDSTRUf4issop&o_+dlZ)uL@YDCPSL6B5)N4vLbI`}W_;nNP%w%V}#Z!_2 zx~o*{E`F4CMh3bvGY46-0Ns408@rx(j#7HYFOZwhAR}Wva&wqj&str%xy#7SVNstu zW{)R7Zts4O*avuVxY+~f%Zubd;eV~OUil7w3l{mW8o*m3KWZ$}m9p9Y{@}ODSzvRH z+L}};JH;K3&6;^X@IC-+z3XpQlJz$$`nu_#JPPg{+RksC|Eg2`A2Dy{tcX=}?=;Sf z(C5B<{tKoqT``xoE3lo)PqZS=T<^OdE;u*2xN+9{tmv;2`g+Fv&WL`Ean{MFJ}V_0 z*tF5)GuQgAch*uT#>0~>OYz%CuEhTrAg`^(nHJHh(y}I6tbclr&*@U7oF%2Bj~0`8 zPi~FQ`B>pQzK&0XHas7X+VT4K-c6&-C&f|I@#V#fxxlG<()WG9S?bc&TdQsMaFkE%>~<$k;aEP4 zK#HYvOZ(#UgcN|o;`{4XG{0&spYV^HcjR~9?3`Bxhl|~OIm%0fZE;Uoi|D94LYdJw zuCtf_ft)T{13qM)_NhMmHY<{Sg&i55Yi$@+U~#84_560Y1mCjsuNb<|Oy!f+`P8wpl{rJf zeva~a#{L+JWaIOG0X=y*cyn~@;RbKJf$RM}&K*ge=%|(0t7c50F<}dMHKxZHQ!elb z&SAXk{*wdu9_m@(*#^%oJR=j<h}ZW z(d%*Ty`=UIcdxf-TeMsX3^}fz?7^=%a`Ax1E<4Vgzhr#U&B=+e&W3Idk02v~`&rtQ z&UWoLm`hWS?I)G)ulXLRO$kqi7XtR=PUVZ)&@Ze0LEe8@)<3+0Z~3wIZ{FC6K_FR( zj2zkSW1k}cE*?N`j{%+lc>8bHZeN9rc^#edNBH|ov6pogTeiUJEA8dp`&wvN-mbPX z+e6rlYB$DR3Cn4380{tcyc%e8Ic-vZUoLI_SvoUx8_j(6-it4k`}!f?X}-&kC0Ms$ z%baImaQq|_;TiF+`fS8*TS}k#_}|9ab~uV}uL9p*A-=sm`1Xtq#9mQ5ey(vN)9Y)1 z^>!c7*q83sM- zeBbHgYw_^2-o;PAj5jm71Dp##dN2IQj#GQ(@?p;kgz;H(4%}Mf8(6w#*I;{%^FA0( z$qKFc>u}$ie!-@5{yth~!K^i3UEy0Z=U#iwY|e%)+GQoegi6ZpJI`>gQnFOj2lEd86K(FV@BY`=+;6mL9KAO8YyAE< z+T8b)HDx2C9h>{EI2`xY7e+*%N`UhT^E)H@Mer|K;@R8}xOU|X-ur3q6Kw8#axHR~ zVvm01)FW(e@|qp*>Ig^J+|&c^C06*>dH))l``hmLJYDRY{63n^U6oMwEBy9s?&Dni zXpBF0aWjy2zXonkmnoMVZLo@s?7^RsG7^8ivBx%<9ASY0rJW9MS9i+KBl)$ z{PET)N5-&$E93K$42!pkq$7$SKdpV$6?5c> zN9XvqI$}q+TiX-r{lomuiar3(C)x@BELdMj=rbPH-yH+iri6OGc43{D1lE6+-TG)k zpYgE%=oqlxpHMIC!WshBW7(6p{~PSdZ@92{yfT;H$Fe79CzSs(zdd_$fMnwq3!ivz zTxP)->$nomtZag(vT)gI;VkO(H%{vAQLO_lY$w zF2Qey&N@TwS?ndTrtHnFoP?~;Krdw9jrC0WVF9wIdNuciEe;hgTFgDN>>pL5Go)F2 z?^WE@W^%_8+Y!cB(6xuU)7bBz&3&_I+rTobf4D;VI#!)XF1wx0^}sW)bZ}T@=Gx0v zOv5e>MM9Amh>4np&O%$S7QA!oy+JEjpTjwbX<3S~b7Bdr=ofqA)hhbR2d*mMQ5?ZE z+ApB}LUPRI{5TYu$=HOWLfRw#<<$z>OCx?GM0Ua>%T@PS@(~=I>}mo|GRXOGR9t3lhdP=f%~IrwG*>~95)-sr5YweYpp z*8Imfca4ozl56eD!(R{}o<496v3`EvR@)Bl3BiAJ|IuyoO#Z9n{blZWJbB~s^aodM zX`D{qNAszkmryTcerHAB1DA=gW{?Y0aSwIyZ!@+b#)z4p)jnr_Do%4zKkH}MursRI zJB!Dad|Sd*9ri{LTx}%p;OfOoSCkWH6_2YXa3#4D5C3T3kN>T8TlSU8$+mXxgy7y0 zV79QyZ0pe>=uYa(s&e)q;^lAf(9S@$Wldm=+jY+!GUJ|tO#_Xrm5k7tP4n)@rCr>8cF(u- z28K({w|>?P9+0>DYWS}91&WYUPgfu_+BxSi_q*-Qi#YSv#<_x3*ixo`>BYskOsOsxU;6wn5_&nHyTM(k26f<4SS zFfh!S!M$^8a>M8GeY=%s;sa)JZe+eIr%ao~h_-@j+44W1p*Az3FCx<%pMVuU-sDox zh|U4`$5Br@{IT*@Y7aBO`qLa_4;$aF=1n@ji2?XArLUITcdjTzzRG7csd{eb1K7`s z?VDU()j5HBBPr*_zUb_OV&N1=76RX~*&`nwgIr$OADIt+v#_H#!jCQBG6K&=u#>0t z?|i0^HA?u7#^GDE4zflZ{3Cgu@C)Vhto0?pdeP_$)LYOd5s%@Im&AQm)ubq-b z?iBe_@`-yDFKQ03HWg>vIrD<4D@PGKJwISA9mP6T{Og1LCp8jlqj8qvGthj9jkUs7 zt`$BJpTW!EHGc0eSw2U{m)UoZFtMEC(aoCEl0*I;@+;-!h9A7#k#kdsZ4MyA8r{35 zn6+a0tiu{Pw_U0-Zv7krt`s}mzbKz}STl-s&MQ%t{JG$I| zKB3-b^E)f5xt~AAx;`6S(7>NnH>!l+Uoh#Av zsyJV*H7JPQnT@>MOsp|;HC*Q-FB*07(K~6&=$(eX=$+G;2hs}z=$)Qkr!v#*r7QB` z#Z|N?eG#6R6hHR^tWTQzn%~Eo+k!{(LNFM3f%63TtxB>08GxOBvtrp459+}rSk$iv z&qc`PJ7;Btr>())3~V{b!gAnN4)!X$ZiNkwDv)n40^{5`eO7r;r5^`A0Y-1lkWL`_ z85v=HHav>8LUS|O7~WyNdgIx{^O3a1&lyK&?1fK7=O$vZ;_3WU%aL>zUvRgSiBI=v zz4)I%>xpkTbZ$nMiI?$N;IO$+d(HCWW*AwH&tO9HseRui!&6=yUDKhfv+fQKdo<0* zu2}?4Goh)mC!pyStaG9%`pJeD5^NsD)Mo-uqk$*02Of(!lhpq%_ zP`-=kO?BIsP543{9JMqJ46qxmO831z47+w)KE zK~5OGk1^J}cGBN?w;Z_R*YW>LzQlWz_!2!@zL+n)KY3oGz(un8LhS#LpWrj>8-B*> zsz&xPAIJw$^7;bgx+^Ev-OpX_uMziI$N8UO%xU!5dq=>}(qCVnZJ9g5o6ujHx!;>O z&UHjZ<8iBp(tNkJCS6_Rxl33PPc02y2I`Z8$ zW2@EBcl^EC&-1@vG_svFOYx62#6Q*&_o%Z5b(|~F+)vzd^!6JKPq!8%-Iq7`%PC~; zSmBlEM2Tz0Kg(};KVy)tn#|_Cd-u_GpYJEsTVj4^MbChjozOR?z%W2_1h8k39isTtng3Bsjao#v+srX#Iqs+*4;pDTW2se zbZ;B>M+tTlwz_oTee%JaNqd>d8s&hD!5dlZ@dmF@90M|o=eBgKD}zH}x{GV{U-0po zIEe*D*3`PPM(f#?Kd_eXHMVK*ISEe=gjn0Gn!@m%8V~K|F<;7vamA;w?e>Wm1g?PX`q5k)RDGRIUz*dG!h;{HP5-g|3A($=vo}U#D}F$KlCRo_ z{uMC=H}wy7+(rA@+z}UIJ)Ok+4B{NBhzn`sj8vmJ6EnK~Bm4o9ujBdu4bD^j?pbSj zm4C$6?m67g@Te6zNZAb^|FU_ee{}mQ+KVkI9a-+TCNB9Uv6bNFBg&>xcLVjx{iEx> zdOEkaLHSGVQSG`TB19kM{aN2RzgG%8F>J*u{1Tpww>`Wwj?TNS<+2@BrjavIUYXJ^ zXB}vv%zKo1o-*0#&Rs1QFdXQ%B3f&NN7*cu=$7T;E$E{o_?EL`gwI;1Sa+U7wX(a9ShkIA5&`;+nHm?Wpg&4czti{xWxR-foR?? z-E~PGLf{|4a}#}+f8(yp8ZY|k^?@PQhJ5At2#*X~x-XY|(dsD6yp2?G4#JCLl75M= zcv~5HH#K)_DbqBkZyd8=w>TqO_-AYR0$*9@BA>5wKkKG!(Cj~IJWlM9;-NllMOJc- ze-Sxry)q}juci!Vs8!}&%2fIK?!|g|s>BcH!FK#s=t2FEpT{v6Yn|n%rN)Wi$_L4kKZNmpUTw#$y%gjk@;~|-Ya(rJfpr2YtF{;R5iS#_aE~9C%}-Xn`s{SI7e2-IkG#?a_|&S2amth z*F*5vg1bYV6QtX2woIP-tnrb~a(EV>=`!i3DqrU-jRf#lNETheT~8}_Irm)&@9}N7 zx^z_A@pkMFP8UA=#;FQlqG3#HTHIPP3wnxA)JE;;)^eTMzD~HNyyUZ2FTPIvn7+c8 zM04hw!P6S>^ra+t(s`sAj9d8nSC0L2#~8cj$hYE0UB{m)SmbMM{J7Vzs(Q~*FPZMGF=jM`BIWQ0@od`* zkoyzuz*gx7+ADn>8tc9`_6P8@Ap_=q2f5?eS=tK_4^?u0>1*)A0pwgAyp|8XZSG$a zjwF{4C)_bN-{9%Ud*{5pH*$5YS38ZYEAj2jG5E`h9%t|uj8@a{q+I1<;rTRh^UqJUy6Sk=JNd6lZYAd#HlD_2#-n{f#;vjei?|13B=OT@ z6Ck36LkiXXaTq`h53TtGJtUV_G+Le``y6 z%{eWc2UDJ&Jv-y#fAx$^(HTXHJH1=D)4YY;INgg<`_^6LkNjC>(GL_iyq)J3#?@1g zbB$Jt&3(4zz;hvI%*jtU)wvtw0CUKT2fkyLyJLRGl5eU$Hi~<@B)f%+!=wVQ>BO zd>^uP@VRG--{r6M`dGBs9vNRoe3Y~=JE5=8_;4e#PB@cX3N~1g;2}FQ!^OYa9SrUg zaVFddZ<6a7#Q7!SC>PjfBV(r8)cZ($QU-mU&^{jBYd9Vxsn_&4D7W7ZMzweY$j2`@?yu7YpGi?XM+|2RN+09SgR zMa+Mtuk4MSe%89oKUm69xVL~k!+dWIW6!E;wH02pI#i4=zpI3Gt(AB3Eexi-boKZ# z4|)z>5^pkx#8b1NTm1jxBgq)?bo~4O?5A8JmU0_x3&6*1@U(K4mTUiI)p*wzM-FA( zrwf1S+=;{6-x0nGa){Mz-#9x|9LejE2WH+&PLD!Q{UUnmPT#0`;{9JR zwoxuW7Z7{jdh^5xOvbXVGXGsLxXL!z|Uq7D; zp1Wzg$a-+pNb7+WYG1!?R~9{k+=;BV!bP{qzQ_I+J;{r+ioLBSd=Hh#sK0DZwOI zT6ms|=T+o=TaI4m(T#jY#%}yPH1+c8d~nr;@WTbC5o=rZbPWhq5-;nqc-&{FFa~@~q#Nd405{ zhc>j`=XlYPO~<|wvZn$BlK)6!~bk-D&IxchCXnFTsSIw;dqjE73U)z<7d$6 zxFkGVv!{E;{Qt0*gWtH*+;~x*5l-GaZ@r zAGB9aI21LpD5v@x;ebm2+mB zgxHxmjMYx~Zz#IJ5Z;aCKU0VH$cOP@I`1;Qci=`bFq3FUI`$y%J!N;9`x>&MSF^?i zcyIh0;6VL!PnB=ehpg};D^k#%?67n1p%q>I7J3f;8tFMdNA<=sn$BnYC0!Cd`^Ja0O@lr{cFytLQf5B8f!%Eyy-W0_ANSIbh+Q#anuQ_R+7;H-vrbHP`+@P|xF z@Skn_l>W1|@&2>D$p7vwsoeRIft=5Tc3IGF7&$sqjoz!iKOE}|2R=8|#CL2$HrDx^ zdn9I&o9+Sj0whB`I<}CHW21j~=f)+&CvWVY9NFj}(K%MM2alrtNR4Bpb0)B5@s=sA zi+?-C=*Wwon}QyEPNVvk1 zyfc<;3f3y_LMhMce5Q8s@21pU(D!{^X6G|Ci<_s^T@vc@QP0mOzFu8M=QB$dzc8il z%D(lQkL!G9@!}V!)NK!KSo(D7!oMzgZpyBE{cH9Vg}S0kMt9zRT7-;U7aIt>3#vr>W3ranqER?3#1-Ux^y$==eOiaF85vLHjPIw zv!cI8ZxD?IM{}NC+z3qUNrmTLh(6)J(&T^*p*ytremkWF-icj^Jk7>FW!|!eb~Q40 z8>n+}2K)l=X|GIkq8fgx-VQ&lPKhAb*2+iF%Kw|otnl@xvggDYwVqW1e>t)#HodRk zcNOE2FQy5-=}G)aLB^F>rWV=al?h;r9=_#DN^+_AUJvBHb7zm+35U_1Yn zVowV%g5ehaQ|`Y4cg&*IGUg0Cka#SWhU{)89x z`7PdOlGpqIdwNOXno5pE!NtCj2bV{$ma$g&3ga`#h+fXx<)crX{oj8nxyNro7ue$X zDz%mzjqfYr5Al5|c=*ai=8U!BY1YQBQD)x;zYnkma_nOH0BU#P_q)J>N48fb{w{CB`%ct-n=`!DSWJot4OyXYYOqwag5XVINazMHDELL171)}*RD&t`M- z_fR){=u|87Xt0M|*jbZ_Id$fu{GM;i=MTPm;06z~fcyJ`8yPLQg~KYwr9D0AS}lW_ zi|9A8vB(Q_Yn7KQEkfpNUsrV+uv?@H_RK?`eb|`yoMJ6UNA4(=yt!!Ew>|v=bkkf+EaTBuWlm&&%G+lYE@ao1BLf1P&T#c#?!{$o@9rkQ zC2O$Jf2}jaEsS?4das#d!16LYrZL_e_wFs;Nf&<(o>{5!c>ZRY%udZg&(*_#bM zYu(sw*7EP5W4&G$3^(ITmA%=@xa(x2rC1XmoNa~gKMwz&`(G}2tK>OeZ|j_iKYFSi z9=^@737)LL*OjE*N@V7B2`3CgG>*|?xC*19si}ccPQTio(hkFCzZF@ zKd(u#*8S=C?CXG^<>05z#la?gAcChlFI3zHj;dcTEhsoYSP}j1kjpDRGp!=x3szid z2P+~MWK|U2*8lQ~(km+p>|qsG@$Nf+u?y-HYwfl*%lK1>OQN49UsignZ&-RV+=^dB z<14`JFOtAiLq5SQ@D^aqf~)$Y?g@SH)iP3czxY1&;&t}; zJ=rSRDsg?SV$M5!==i;FqV8+-@oD#DKdHU1(B7s`XwUEozMD$?S_|;gF7nwu`{Y+Z zBgq_Z5Au{ZSu^l~ErRE9{;ku0Uj&z?jJr?n?J?!O*>{gAe$UctZzS!>fAzVsPCw5> zL(OgNb)U&Q{Ws11ZzKPG1DeT)niw}D9Nz#hPDB$+J@1m1njq&$vUy^Oe-t#`pFXwp_mM7vD7A*hj6%dBwrbXV0w_ zSudRA(?=b;S71nKaWJ#ASognajhO}As{_^q&aHetEt7RD4vxk^peWFPNKrGngz_0z zgqW(dRAf2lF~u(dn|IK_Z8I>wtusF*P_%z4ar)$Hw87nh3-Lv{I;ywc>05l1 z>#ljHyZeqBe--1;9AXz|G0qUb<&&SF_s}bbZ##?MIlyGrikuU}+t0U_tK3lD8ygH5 z8h3??H}0~EGi}S%3&4BO@5p!~=5{CRm~0-_J@Nw|8Q)~a%o_B~)2!vv!`^_7dZ#*~ z*@;G`>JIEw)}FrmKY<$f5!j5ckF~WGdc^O^bk4*dHr4fq(Wc~Y6X&9CdBs|OBXj>l z1+u7$84+YtC za`__2IT-1Zv*DHf*bLGW4e!9CUYS0;Ij@54IG1|pW zmw)lgv^$J;nNOF!kG{Tt4(rHN`K0Y<+s-`i*|uza(o673nTuXoP{I>nD!-vs?_b>5Yd>CK$ zcf8Y_e~-2wp}h3!c)zSC)3i_4h~G>52QBcR=vxYXJw49zYZ`q)a*e&T^PyjrSuca( zAnS?t$!uu9AP@eBhqP8n7q4~sAw+rcTYQ@(|Ao&`^sJri($jL+T<4x%L(eAY>1Tfl z_>GNlp}qXGN&43PTX&GlNqyHG>OVd?)_4_-bN^oM(_oOpkX zbDpi1xe{QV(wY>+FBq{qpD6(Dg4|;6ZcyCB(iOJZ-*fV|*vReE;PFbu9Pq4|6VWz4 zCgx~;ZPGeVU(SA^6DM=c$KAu#<|x|CAvbi6&is13)Z+3|LveU@f9qB8l6YS-=is#| z_)Lym%M$w*T?0ApSlI(o|3zl3MV(87y<-h9#z>OdTGrb2o5zPA$}24{;T*4ct0bMg6yQ_qZ7aNa0(&nHq9;6bTxs!&VWq`CTxKoo z`p7Ci*o{AM3-W0yz7Y6Wel68)qK!J*lP*8DkuST^>U^e^c5>ibE2r4l?})-W$@HtSR`@OSanJUVJ%`U0dD5H5 z1iSpz_o@GPWEXsYZ&jlIj5b~}ZS+0wu5;)b%SM^rF{Fl zPp7&6`zgy9F4NhnYVzb44zSZX{}giw%`;m{_SSs%I$qRV<%jD<2CJ8 zp6Iq)5^klxZPc#=HtY08WV6oVg}Rsrl{_H}$N+;^6rU@xpbN`_VXOPTe-^$6W9M>)g<7jT;*B# zPqEVkf7RU=5@Xx9F1U7ly>a9D`Pja23P!by4*a0vVt)dS^iJ(HCbX9e?Tzo4_ErIp z^n?{|`O4XLPkij(m|M&BJ}x%Uj1T-9xH)Ij1NWe^PX+fKN#L%cZXf|haNPsrO=iAl zM&-Zw7Vt=KN%RE{h8~txYxWaUd2Zl*68i4;Kn}M35+@($<;=-%ulAXII?IU<-9K3Q zg;Vxf;6py_mE5aTO3Y;oa!Gpd^~XCgm$4n3t4U4SHx?O?Wnx5~wMucKe)?>GjdPUX zI)t8%zSZ?Ud|~E)$QL$yzm$0xJwyBN`tPrlx0vTP_21;ViLpzTczfO%w{jLX$J(I% zv2t_^$!Skd`rs$DU zl<~;+^~%}6s&TCcF7fwQz`5|9%>UY&ZbfFMr0iQvZm3=C-J4je#iefq4p;72kQdre z=L<{^f`9Iyoz7YQdHIx6-&NQs2Y{!5G3OJvD;uIDm3uH)v*$5ytFJEXgg!$9?Cmr6 z(Wk5#5$5>|%#}jsVbn)ibSdI{yCkn;%=Zt7r~MH5rt?s*le^|h-ZfAMS-(zlqmer0 z#M7t_aR9`CerRNCTbvv$Wxqgj(2Hk(m$n!9d=JIr-t!F$|H)+Ue9=PsSY^H=(=C5^ z6*QX1d!1XFJJq3;_=RzWe{_!gT^Z3d@VtRC($nJOOyx`3HP}~BW%;JZ;MZ(BWqvOB zi1~emX|(~?$WYgG%QtjhZF+RvEa$$m`on9QH{v5IWo!k=blp8goT}M#i1qhzN2B%+ zQli(gCnY^Yak8FH;`uDfeL8zn9B#W@acJj6+NSozqhY)IjXyy#E{&}Dv6Gzk^B8v- zxgCuEI+PM_ox}Ms`m&&No$vyWnsJY{!XK47<9>|Pcpon&zOR_p$%!Z zrEgTgONDmY^qJhb+YJAt@!v(fujl>6cG*LV>;dz*_jJY@JAGc0a-ybQJpHcQD(`*Q zPEY6F`snAu$zJlUp7QbD=4{`9_7>zyI`>tm?%Vz$(M8;Ip99@bgEsH=w>EUsPk?sb z_79A9qX)is$I4Of`3Fzmi#{B2evc6X9(VQY{9qFi5mfC=_BpHRexLM&o8FzyF5Q}_}%86_JH=c z$@{7^dVBhZHWU*7RRJ8_zn5M}zv=W}dG*}R8paW{eGiGJ!0&e5(FP8UT#z1aXwZ-v zZv8g#IpAA#$bk+wpKmXRw#0=%+YIz~&cEf$=CV`!`bVgvxTReF%MFpwo%)ZmhKo-7 zFC~u0rB#)DIdM8bsfqK2e$c3pF%yHkAst$h`^X)0FnS&MdV8?7fjk29v@R~I9T3$T zw9-zS7m`i}{VMDM(E;E^dfc+ZjvmLnnlaaHa{z5Bm(ohwsKuY<-BH(rgLJFQ*whX1 zA!FHfrgK(NI4A}O;={w_tuZQ4SchhR-b|95cIugA|2 zKt^`U*WmdXLj8%x0k&y0-EN2UjeDcHTp#rvv$;TZAbt?9nV zU%@x=C>vORy1KhlBi-B~*IM5{@`XRJUgRL_xjXX`o*R&9E&VSb4!j@vcQ53-Ki}u^ zox=C|e5dlQIB_4}Q~CDu9pam~a}$4Vuh}?N=P`O?>@C{1>EpvI;^rx1f5G?+@Ch+T z!=nF|t@ILt&8z+`G867ouJUW6`Glp5r<6+=WYkBWjGNPxu zW0CLwJJ94t(PXw2?gxHq`i0gsaQ1o?IH_{!ksK!((c}BH)4P}d!BwUBG=6Jx-Z*P& zX7oK`7v-J&X<(bn=lT z!;jj|q;B2EJDY=c_8A47(=NeJwhDX%4}6z>^+iwb!7g0-v2xOJ$16GMHX+l!7=(DZ z<+sg2CtA>K{I>VR`#@(QPr>VD@?*)r>H}8CPn#Jfj(PhO&axRFAN(!9!2@xAd-O)* zxBn{buEH)7%@g~Vz1OD>^Pv*GDY4v)X2)0CR%S1+NCIaIzS_uT#us|oYsk@`@SNlB zd28?WFT7i&cW&-1y_4N<7n8?&kl! z4!qZagE`!T6+lObrCIB=SBB2p^&w5_faX&<`o)1F58{_oz)oq9R{-+GGkzA`<$3ctcHd7pz0mm|4u zrH8+#-+FFH3oqeWe0U%0oboSt^pwB(a%ks+cJdE-IT3QfQ2_r)Fvkj~rIdE=!H?KR zo&?EuWH5fE`1}aH^^R8O5n`MA)=_SORN~V6As_m)e-|KbjI#>2aQEC2UpS*G6v_Hr zX=F-eC~^(>l^%7C-zqNmhlBhx^&^9=^^zfl;lP~5U{)n?0U`}<;Whv>(on&R-1Zh?G`#R^*WRnq5^!HI^oL`_g1%Ay1blla!0VtGk=KrR%6`_JQ2-J;<7dxrXWBS{}NazlNLp^u`yULV7ZEV_+$7tpR1cP?JOd-eM` z{RWcYCiG$VjBBaG{5G(}<0p^*t_1Hh`9IXvyTAB5+AKnLJbYzoKI`+zY z>d6LPFvzh5RW~V4=uO3qNKzy0zzGQk1a2~#;g84U% zd05VTypVZ$0r#+Cvuttp$Suk!mt=MpYbb5$T)Ft9incr-lgg|_my^CD*xTLmwUmD_ zuT=XGhPPBE89%LJo(aZPz}Uk4y^*%XQ-V=#%eUJ{PPPm%a`IN}gsZ`Uu_tl{7<)qY zlu!9kkYCa@%zrifH`TGV%zqYpUzXLSwwq}K+-KszU#EGX93iV%Yr46YS!)hwm$$2( zc)8+@sivPDNje^Petg{>&mZ?Y{QD^5kPQe-+iq_-B|M9{wrftW4Y(KH)%x|D&#@=f z(aVSTd_uil)RPbEDe778_13%L2)qRi-cDuAkExISy=5DYQg$q5wYL0F<>Tf?Gv#gi z>_o<1#X24@1CDp!lh@h&@97Lz6F(?8RY!Q$KE0k3`BL*s`kMB|5t+S$^A!I?=A-BWV)fz3v6fq%x1qc3qQ4dV`hJzb65vCdv+tTE?&>u z1P==i$*)<+{JcXuo1@6qeAd@MYcTBbWe{J|OxB6FfqO3a&~xhmYld{e0QIHkJOPhC zkz%cvA2?Bulg%)bwIT=Ia3*t2>ylS*tXp?0cU{g!j?Cl^=~Bu?(H-X!3pQ8p;a$C- zg`OCtpSj2q^`n0Cz52))~k{o(F)rXBR7Jv{5(bmY>;?^+S{m4@$BZSP67 z)>oj@XL{1w=Z>KPwr*B`HJbAl^%|uD?zVq7Wz{w zYqxR?&Xev7f45M+Axc_M7(LzC==#8Vx5p(AbTat@`XEjmcMts&les+-lQh@=$Xy-tJ1M&zlgsZ+;#h`u3_Y= z6Te@@xb^MD>7BN%`$6qP*6v!kL%8wbH%P?WXSZoT?#NsTG1@V710P;{qiIj@IPZr? zCy%t28@QnTGr*|vBTLaOher=4^gDpMg6&s~e=g&%<`eIG(U`3%`0L6-T|Y*rto&Yi zgdEAsD_8r&f7kETDdETU8=g9Wvp|*oE{Z&*eVM2e?;dbt?Q4N$qjE-+rG^{EpxgZv z-8J#Q*{R`+*e|GDoEjbi9v3`KY~<6#gCIMk3#S@9_Qp@X^?UmHFSXNmucxQ4U8&)- zK0{y6A3Y~iWnMqxCmbKnX`eZAKS*_}WS_+C-{Mbp#zNgZ+CLlIj~3p*)dZgPUC4K` z7|pf7e2QB}<9vC9wY-4e>2Cc@_rD?T_W<`h;C_4i459-%tD(c~E*;Ky>9ASf(BWd2 z4)L(};HE4!e6CA}r2o%O4Yx5i!H8ndep_ldpKsCOba-Va`qy9hB<@#>UlQ-wkqi>; zQ$!2ovSQ;*jG5><)(V%AN5`|ZXwUdxg~McJizq7?1_Q%?(SNe}nu|>DnXl*;=-Kz` z?6R4&@TtiYg6!422;4c7eY5YFGwmLnRp6zOIo$Ie`#|ukd@KAZA7|~mw>ondA7f%Oizew9!L+coq$~NRVur^R8D`bVY82lrv68awvPiYRn2oEi! zZsPns=LN+gGnbG_-ds9^Gt$DNCqvRSm#{Np(4-QXsr)NW4i{%`_{^A`XRUw9c)RPB z-aK5AYs27ewb_`^=6jS=-;V(A#lX7>o1$E6bes=dw6Iw3`~UXHeeg=o=cA_{b055G z3-d5x&viHRLU7rGoxEY*yx;l?dAihI+<(uzFx~)+$#`s4+?t^Eq=&!YZ}C?HYl3L{ z7yLZG0QVcg{Umoi*s(=$^sWa_(3ke$YWLi7RGwN5jEQ{a(Ba7S;6axT&U%m)T{qlX z-r&I{UaGOeKSX8-KEq3K{nnXr4~rHt?!Ti){*6zv@e5z*CfhUqI~3W;o^KQF4ri~i z^5&|}^-~l>y}qI%GoIexN1HFFxODP(uL3%09mN(m@dw`n{yX^S|9Ck5BMy$lxi!L!KQsC} z=GAvpH(?IULf@$ZZ}BpS`RB^F-dK;5MkiSd6YAB*)f+w9ktY%Q{X#;&f`1G+Pt4sm z)Zn}~C!04P|M9YTy;s(F`0-@b&X*neaW{R%%b|nA92g`&ixOZk>k_dmG3XgTcF7sZ z_R|gg8LS7TM=b+RjQj}PdGy$n<9o_XJ!(5yl&Q?K!+*s0v0(9eor@NybWVVuBoklS z?2N%1tA~fxNpOIB2wZ`o)bu8uX11EW599B=oOIr=zuQ%2*cK$ghQ8cz!k{sk_r z#zK?r=;(q^b$;N|>NC85iT9tB*ASR9<&#cEyUWqrG{+KoUFDCa8%oyy9QdmSm42$2 z8}^{E1N-u%>K?28hYb#iRe=}p)7azYqVRAP^iP}<7a95=IVbXx(BG_AZVt>*E}mbY zuXsA(B!_1+qg!}?eG(Wf^a}X~l~ZaZ_NkXwvG6ExSf2n#=S$Aqd^`>g#ocDOaIE6} z=MvyBHU_k+WDdmQ;_KS5DHG%CyttCe*;e>e&Yx8+?wdo@lS%sj1MHwG@-LRtZam*? zij!L&P3Mj2OJ8GqIx&H3%bJ{+z^&*VvOh&T+3WgF>_3-vU-Cr#zfS+Dl*`n3#rM4X z5pkS?=~#SIKE|QhmEbCsx<^}o{~7uFzJn~1Kep~utkbnGCD%bdDSzL$4Nqi6^9)b? z@ALOf2i6aOwFx@J&m|+<;U9cI>?xWlpY9T{?}!DWIxp?qUP`4imvv?Ix!ISoA_UT5`q zc{pv0-{Su}&TVH?l6Ll=odD}QxAxKf{htJ@TfS?nGxryGZ3z%&*AIDR_yxrjdLf5v_6E&PtQvrJ#eXTE>U zZwnaI#s~fZksbJRnm=B+qkEp!6`=fPum1-1@7$R(m;PT*>_4F{cMGhK?ObU3&L3vY zICF%xu2FsCCo(kU`#<>|%Cgpduw-DQ`$ntlgA%Lj7sEB~6mo~&@^p-xEb_y4f(Q8$ zV&7&h{7goDOaCj@>`tGt#!9u<_=AdNxr*Oc^4rgE-=^CIf28^?$8m1%fD@Z} z5F5Is|KLb$I_og;Jq7lJ&gu!B&qyZM3<_M*@+jx)r`zmLrG?Gjf30tF_HF*iLgC5H z#rRF0b557f(Ta9nXpwUWSWHaK6@1&|bhL>poY2`go}4D{6T`D;NThpCN^y6+)fGD% zpZ{TE3-Eb<^fPPuN3?MWm^yjxQhPk_zqhpb;Aw)Pv@14P-@~IHl2>O~YG{L%YlrLh zc0X9>vo4EmNiW7nH@}g5AI-i&Itaob=rh)=b<+Ag%a4qj?^bzjc2 z{y*%Gl-T~x7cLENuYO@|NASrV9R*pjj<&1pu8*#@yAJK}ZRq+Y<=cH5I(Pdv#NP95 zXt~d~q4h!EhRMPKcBlB}RGw?{t#$I1xAX`1iZS{X`MF1czgtSX0+Wx#%YO3D#IADU zYd@UI{tIhZ%cH(9zK*rIHG{)HB(F&9no!q>g%djKL__%aea3KrG3=j5-8oL(4@yE^ zofo?G{eGu@%i!>O)xQ=V8Jbz2aRvEi)W$r)Q2~z(<1?Mla6aXHGWks7leOmk8)^HS z@+}0~Ti+(`caGa;Xh`^F+MJENHgnJ#)H4SnqCumrdO7;>}ZH$2)S>d*6I~=QAcp zl7H};mNUr~OG!O1O&RV{IP9Xf+e|>95K6R%-uYy&B!;|d%PS&z@ z?@ul5`sl*au9nAvV+n9v4&SE3L-20*l0b2%=rtTZ9L?{O_+;}L%V%n^xcfA|mjsJV z99u6WzZ&VP3#QD^mAxPj8IGez?xQH=^A|LYg(4PJjIMzwDZQ?G>Ck< zX=dDS7o&eHX*x$dpHsVZ%Kzi-+~cDvuf4zbOn^NB1SAPJl>{s%;H6%;#6p=Qs9d}i zsq|>iA;38`0WXJ&T8l~oUIL-jktj!LCBa%VQ)9gZ6>Cc(ttUur1zW4NmjrA(5!;Gf z%vf;V?{8m{oyi2~dC&X)F`qqq?`J>Hde&Ocde&Ocde#%~4~*5+zrjo#-bfpJ2NcHf z|5o%R4=Zw{g8FkSzdbdt^8TrD{J|BpZ}c%5;?s)~9nbUKUw{cd+S`9t1N$Ny+I~bo zpAYkFW<%ScY&%!eEM!RgKg>iCdFtE#Q8BF@f8(BhS2T>F9?>n2TxR&|8{#V%XDiLb zf&0uvVm0+XZYJ8EgwKBnTt8r(K7LWOeIIT9-Ar5yPZ{SsWA01hHP(%!D{!`DB*614 zn70prLmPjC_+}_|fa~}fbE29DG6#mL_jorIIy=>IuPIc>0A8x?kq}YeI zI^oTT9I)5iG9xb_U&W^_^fP`=Q8(MZ3(ipFFt(#`t3C=xO`UaRMsDO=J$Go95t+&T z&79#GKhWJ*&{-e!`v_bwXD|3r>Y7K~hjazWaqqm9BG-mvPg>+ploh`{?rZZx&H%RC z+Z&AJgF9^IN%0lEw6{GNA<(DBF26V!IShR3@po$u`F3ZW!N@3RsQ4+-U=sZi9yRA3 zSb!{Y{j0-W|LVU&kGSgt^~R^)2RaOzNJqGpwcaf1Oa_MAKGZB^Qt3Cp(Qz5yciJZJtSA=aEOFb#gfI2LF|N4W0vUH{0tcT{sRzuECcX zzXJJ``fhO`Qk%^?>5x90cYcffr~>EGmmh}jF9L?%;-zi7kF%vmTdu7B0r|?GWDo5y zCywv}$O^|dXc$N{$K2a##^xKG&fzq{u`|^$=2U8#)!Z$5%PQ3dGqVB>AZC$ zFK@p3_Fv#fc6XWI=5yCvo@~+iGhMkhQe%~gXUy{U)D@r~sg5evBT8jwx^+BE9rF8J z1P%q$;b6$l?QgGvagRUSj(c<9bMpFY-(hIPoeh2MU z_u16_d){krQ|ewa<>}I1_@S)z-OM~`5hrN=hjdVL-zs8mau$XYFFfYN-8P+;Reh+x zk*EW&SO3-7=USh-&$Yye99a(E^mA;F@!-9{9N!Vm=y)+3-QghbrC-J8o6$X#A#Ue^ zyGw`I`w8-Rz8AP(Ip3&}{PpTq+=}{J0X}{YPulah>QBd)^nU_4@$X}z-}YK1bI}9E z25o_pmEc43kc~57dY$rS0Ldx@Djdo91E;{ye=**8>hXKZ60`%H|yK;$b!w4UnFf3#ul+cOit9`v_PHLA;r zXAfb|m7}AFvbm1oI>KTUn@b*6bJ)^c)`-i;p_dLd5^*;t_SeBri27%#{yTPT4BZjhSWuMr18ZW{ zqsm*5Z*Kan`jl^4)n&x#-$MM+ml>N^@ZF-b!->Uu=ugi z@P((v{}Onv{*gyMt~&acj}7y>$9@XEjl$B?Oso7fT^Hqqj~dpZ({l4mPb(Y6G{`=@Ge3aeDe!~lm3(CRM_)yEpSx;D_3i=xt z+{!zx9Y2g;;t=)Me;c1`rMowYJlx*iqzd#~?KQIZCb3UTdy`DYnD!>6?n7G0^J;ir z`<0q5=zYIZc(A))iFW+^mD-5Et%81*dD**VL-CIQ;|O3H$(|-X&*Aw-uHRrD?6?5g zxcc&U<#)LL6{BV*^MZe^x%nJBc5rTfvh=N)L1(`u^xL^|dr9Gr%y%9guRWCP1+*rv zfyROTA?6bc`;OS({nkC39u*&rA9xwDUe>;u=T<#Bhi@kw-PH0yrqOt7by#tfjXRLd z<<*=khrYP-ek1Wm_D}}M19K%f(D$w2EptH8OT=s~6V3{*+f$aq zo-)}B@c3@~J!5lWNrvrPIAecu!*85ef^PK9%Ojp`_2nH0e?x~-X20yQWW&;ie^NGZj+$fx)?BA-_;#8)JlQZS zuXow-W9ETl$p*>qve$|V%gEmlMULk$3`R1EFP}YaRF^GY%W8)-4jqFb!Eh#HeQ8i7Z-WL@tAtt;9G=r&3wX1JmHw zQuHqA$gR-*)2F)R*(dt%lV=WXvu)bG!}_F0cVmBY3;Csnt)zcd;&bnebmFzEY`N|6 zg(FA*1Yd|JyzjpTJi?`o!-3(1_k1S&R4@cNz?{Wi(_ORwdIx%OW;Ns3hCP>+%d_Q@G{w_RRJ-r+TB%j}wWHxnP5I`4+a|E_sAG?0 zYaBV{heq&4*GR9Fo@ny(bj20e(z3Vz;kK{mx5{6$+KAlD8B^-7&e(YJuw#esr=0yQ z@0apkFv!0W;-~igb^kkc%l3VrZ~V{y&9>js|7W59^G^HikK7bQ7fYWTNoSC1Z;!?i zK99uApE|jQ>W~d((0LQ|mJVOLD5(5=4=QI1^58+uua)pnU9l1QWhVP5u(Run*$aaV zS;e|YJ-OwrA!fCaXGH3XiBtSDzJ2s9@oANGSLR&oqAjYkjQRF9XQT2jH36UU2T7mTny^jJrXi6` z@bZ9z&u5(dwb`+sG&%cg>)FS-mE27APt0s0?*cjD+v(G7kjd;XO0gCX>} zNw?>Blx2{ABN%w(7d%&7m&$WsUCry<8`|@idvc_L{%ec{xGyd}oqA2`wPJted9Lfv zY{f>bIM=9=Z%@1@|B!qD@{@Y^XS)9dmlxmo5OhIDnkzbruf@+}+;Y;P?0FPFl2?9B zn(~@|opM>Rr&tG6eagT7Rp3E4nl4Qx;MH>*^@w&F4~h%=6@BsQ*yzReBMs*nXk zEu#_NQ{pf>ZObv^OmbpRvp#nlWu+HdLG%dv=byt1%uI65VAJ-&)?vTPUU*+P$YP9C zAge3k>GF^xm#c~$xx9zHIy&>ilgpLJD?i9ZL7hYyu`g?V@s{c9z6iZS@? zh&?W9hej&#rN7EM`7`aY)07o?mV3>q_V2Cr`6xlD63{awpB zq>2lAm~o+d<=2Yy&hLX-2#?86RSdmU#^clP0++_crOY*gS$q&r^_#()7uerr$BHSp zA6%r1i7hbTH~e$b&&MziVAbXCg7e0QUovXG z%sjb;oV@k)PkAcy@CQ8nYA~|t)sn)hRWl09UT4m6-!0@_#jWAQ{d{NZz1VFle_iOtvY3k1S;o&&QQ6AF~b`xu11OZ++08ZzcoTcz2q7Gv9>=G#_@83#G&C@%_pSa+K0;5Li9^ zIJNC&w{7X5YX2tM*I2p}8E_{3OpPZqN{z_Ct?(x2US~0`yx1|X52a&_$X9OSy&VJU z#GH9;s9rA?>&Bto;|et=N#E90b*ICR>MnBYPWN8*f5jftS+UE2{|Nn07bBPj56|o4 z@UQky-h!8~VQoKWA<4 zWCGZ={4_&+y6YbLu$(^JMjx7dc-TT8hS7)H=)Kw+RwZQ_{UIwA+o&<*<1?0 zr}p8(Ui#1<9pKC(x4#w24@WBonkSt}`ytA5zcjnx(;H{+23nT}W^W6wL=hkZ_j_fGw z>)4bAIA+~)w=+(ivuF;_XCHmgu?LoawxC06dzwSa?0MSBt5U}^#((2x=r5i1$IZ^1 zZ3dnFL|eHRzi90%wd`{#WuNB$0Bbyg;j73WTP~F-&zd8b=JTvC;=-!vb<=MPS}lRQ1x?BIT0sCipQW~K4&blLFC^;O_9G$On;`(KKq zoVAnK!MeNT!Z2I5I_r-fT*8TYqvKoh&K&Q0mlYdf%WlO}Di3haJ+k&3@yh2N-LZ5; zB#$_f^XZfPDUR+qB9ia&g!*K|X2Vf#uVXiN&Qn>j&%lrRD!;qn+7C@^*$wQ{Ld~WJw=#N)N^^9fCb`GGkyUYj#=DqrpDznw=F( z!cRs9@nM`_WDRD$9^a=$Uf1EZ&i?p?z?%(UTn&u>f(};rxE0A~?iKH5Gv2Ng?9!*d zdR`<>e!FJINuTdxznza1}CWQV6*cIVEG$>6|!f9x|;L_wf}-W&K;Y63~;~Jl-y8{51Y0V8IN!V31f3U*H(NhFX4x^WsPz| zuFQzaM&``ai0tBd8&n3nOyiY2b!7uOUi<`{ALiNN3Cu}6x7QbROhwV0;_SWftKR2e#EgvKXmpFC1ylDUf zeY%N$?_r$ioL%&@H}B3JX|K0<=k4xg9B$g8wR|%c#wQRzhjjycY~0#?Y%De-zL+?3lEu7|w%k}Fa%cc&!NUKNg@0gOMY8NEqbq&AdYUPc8+i-N;>rJ~c1@j=on}eMX*F&pQhpB0tMGzy7V!Q;I1k zAHei*c-bnu+<>dB(5TLGT7NA#v+wy)%G%%T9X(|c-$)P4zvJYqtbwO=FFRv^UAFT( z%G=)^7(Hbb-_;E@cHIXqrLzOS?Gu##!94b+bdLuz%&g@041C-J;k%RIzd`WfVEAze zvif9lVG+{?Jw=xa@-Ta2Rp)X}q5iiXBJL7Djs6Q?4gZDB&sf!cBje_lYf7S}CpS%7 zq;cIp(9kxjV%ni=`$b#(XEd};VEo@#+)#Vp)P~ya=B~%{jhZdj7Dii#k`FQX=u|!H z$1}#;?%U-%g3s!q-;|-OeRI7Kn$L9S0x#EA5j1ApH5S|J&ckEX+sMrcEcx%5(aFfS zGU|zsX8(n6eyelG>2UR7uMwF7Uuxd$IsawFPKK9`u9ypcz674G1YcKxx6ARlUPi8> z3>&{%cg9bKZ;pH)bt7FJuAiH$($bmF3H7aQyBV2zu2EA2AMc&!jA8jYQu}ZXeHaOB z!s!&ZA6^`y*^G=mcs4us+b`k=;hFYrdw455oZRvgWJrAx>#Y3rP1^AlG$ZdW zHEPtB{PMCpRrXVSslrLxu>;+I&NnZUZ>|*k7~d#x42}JY>lDUa*|&?++JY+cd-OQL zyqob=30$eRs`gIiLu+(`Zf!4gb;0j5W-Vas@!q9rkoslE`r&T&gl<-<^7rBOks8+*bj2s^6&B4-OGj$ z{`HOc+WY3$KXLqHchi0<&RBD>@vphTu}Pi1SS{48aV$8zciZiHvSMFPQ_p*6gF9#* zr(NddgqgB;s?mn^82h~3@-erKe@SUWG>e0$q3CzdbIoF&ucEzFTtTnSetYzKD5Y&L zW-t%GvhXwpKd$;&?Y4bkO52iY7J2yne6i5wixKd$`1oJ3rB%lm@R`cPz~sggcJa{o zK|Og$^~|9jjZ@*?yQ*IN5ksTNhkMdlew?RksDpi?7Wnh`#j8jD z*C72-d)FO@p5YXn3SLhi5FeBUy7R#WE^hZ00UKqH2S?iy2ad1Vc+HCan(ywWU#t10 zjvs76dyTXfe58&C(e6a;*#0(*@Vjfr`s>P|zux}N?tH_#+K;>2zX|(3J0|&=3aj9e zy?LGJ$zJ1LX;ExMe#cs`e5Mw-RQ%kpq1(;ONj84rTggJnh=s0zhjI^}hMJ4Rcm(g-aiA5B>3Ad>$CZH`y+omh%0`G&ItkW7s62!e0dA~uXkyk%iO!1zOyF3zZF|H zouA?b{7f(MO}rD2&YFLli^n}S9&=)2U7C60b2Rtu=qc^kFv^?S9X~m-VJYQ@axYk9 z^EA`1WJwTzTPFG$vB4__;A;p*@DIFctUEt)0KFaED^eM#nEN(*D(94MmF-`L4IW3g z^lX|sa^4NV&eZ&!YZ;SD8Bx1q5QUlG2^yW+~8 zIQJKzQ=Q3N(7L*LYHjd4Q`^vC>w>3tJiq#@9R(T0t%KV(?Bucl{tx^Lb=;R>t52Uv z9zWtJS98yNl;ypzAlK?>d=g=}jW82ibKrksJhXJIF1;xCwuzhD9Wf)>`m zs_zQf{(|L2Q#H!_gI&g3~$ZytA=QomgbA?xN3< zjhXm2s_)u?f204teei_WRvG@XeXM^~XJ%C|#lNu}Uo7qIT4`q5{*5BqkUdnYbMFTc z$IM(7cXa1xth`Z?XfpX`L32T*{*0;7nWpi8wd=G<`*y=VbAs_eEaWNfSrdJt?a98; z`k=Whe!lG^C3fNze5+>UPV`~fw|TTx%lnQA&b}=BdtVzmr^0@=a8zfV@_iKWo$6Hm zCT-@mj2-nWh7j8kH12l78YnzM}T zF>IlSRLq@kmm`jzcgfX?P0GOj=z|^77uxiLMizWe9%JG^4hDnvxsSQP>_4YY^%3)C z$Hvr=FIak*`lz$&*5K2zkYyo!Z3UcH*G}GL)(5+t-}csK`Ao9$nczEq+>sITkJ#<_ zVu>W*E+hu%3)D^S);AT`^DePFJVK%Sev|smLa&^OKkqTS zPmUjGG{=KUe_Cl|FI~+wmpT^k>~+SI#`iWXRlak^rO9}_;7qfoRclO~L#lK6h+ltTG3WB(e|q3<1PpA%<@$HJ%CbDPFQYVN$De5148EY1qCdF|*X z%X-tn)7QX{g*{%D>gyS6MArA0z3u2eUjDq}o&R<(-v|fOfcHb}>U3?++Di^xpFc-^ zVt+(88&f=>^#*MD$TAy_E+0x9{Dz?Oc#|CR%n$y$wD$^s@H2T2j1*0uxx4AK$X@D@ zE+U`IoD8jfbk${-t&rc*dH-b4wC!1gx^L=hW;G)}^ELOU$su>^Gd*+2{qjQO2K1A> z@!F~X%!&WBa*aaG?PYxDw;Lssmf){h&zz=MJ1k(mhGapXQs3b5BuPE_-42bdv_i>9^Cz1xGx6oL+Hn;c(*>E>=-?b z_#Ca%VV_Jdnz@hlxOt+3t1=~T=oE^1 zRy?Bg^IY&*i~PHcI;Vio;U%2+h)-7dM6R(txMJ=~aGA}TXueTP3|CkD^C8Z^kj${w zUEZN{WbD1be4|*hQu_aP0lDtd)LY6r*u%h$PG{HsWzGx%{<-nwCcFOBI>q<4-c~z0 zau2@nvTqXScl+dyIC^#2UBpX*Cl6+=`IWA^tRvUOQ99e^Hrj4=!H&dX0XtS;a` zF>Gw#GqT3%{$H=aej$qk?No20c*_8>RTVVthy%mnd~aFx$KBfZo5zw9&NNw}+bak>zi32*lU zZ$-d3IaNN-!AGh3YT@JQ#5)+Q^Ugtb>Y1Tu1Brp)I)^vi;^sPseK>m7 zL>`RkBO=e3~sunjC0L@1D-{*WInV=gU?#d z!q_&&q1g{8`wIVSy?p!7N%7r^QTE3sPua>^tD6JqZ2nic^Ikr9@y@IqJ={o)ZZ!(E zZag#C@uGA=?|mz4{|8w2AE$b-l}+ZzQ}|yC?pyn@PHd1 zebb=4@@>3{-edo#jyL!(JyddEG3GNhCeguiTp!2_jLZMPNAmCd=ByD9wEiW=P^c7Z5!`=t1$GJ-BHT54I9Va57FO9e6x`;W3QDm7bv!6IC~K& zSIPWQdkMZ8`Z2bRd{N+NEW8A7ZCOO#2Ks8pzxUQ&h1*`O+FNa~AKcjLwfnEM*|wTk zSL#rk>=obYwcAX2wJW_)6NSCa~iMRuZGcIveu#fxK{Tq;;_o`7X3b)xnB4;bSBQZQ&SWt6z`A zF5T+IptHWbG#A{b+7gPjY7ZLZ5;C^h^bH!366*747P_NlqjiV1(PH0m0sC5~6X$Jd z?=s^vUwfC)Z@n?;*%aF6)7u`O1K?>GJ`2kj_Bwl~_w8Umo8-OLC9~rz$m6!+t$JS` zx0&Ete%-IX#GV3pQ**6&`a15xU!(L3WTN&(JWyN9MtGb8t)7i9% z)%ap5yJb;cB#Ez2F_z*j%gCv23l=q)rqMu7vq%T;-d;t126AGEA9cn@@^PbZI1KF?-;jdOaBl(b~w)SU?;RAEy%O1!x^6dDsThD+_eU0eKyUeKK4%6*7 z@%Eau2G~nfHd(Pr4-_}$MNDE9CekOHM)cu*cuj2wk4kSI8v8Y4?{40&=J)>s|0k#5 zzu$52|FS(-{r|&1gZ;Sp&-uT^zjQ1z4eyXQodt|*^K2>;(&WRvEyiIr;~-Ar!o(9v(crSaKL zKEh&S)~DDK^N@kNwQu=qVAnWnVa%*S|F*F0+Tn4@7s(pMinRtgw`MVWRJgA`LprqM z(>36K@%%%aapc8fD_4^jkKKA-II1?L(Z)Xo^T&4r^BujyEFGsC%;;*zg8AS62QUx* zXJID)W9aCLUSU3hJnsfG>$b;&`P}~j%zJ6$pTaZwCPPQR;lND3$W-3=Wh%_%Ank;i z7@BUp!F|U60Ol8Ln7d+r?EP!VxIBDx@`)~j=fX*i6C-kUNB0;TzdtD5cjvj^%}n{u zKMr*!5Vl~aFXqS((vQjAFZA?R_{Z++FF51#_LOm+TK~tqx4%Pf`{M>)PO0DDH;vVw z5rcsp(r6e>(-d=W5r@e9#@MWwhW_w?)jY`75lpk-CE;r0=a{RHl)l9tlx61KvH>k@ zi!Ohd!8byC!)l&DVeR|!`?7c|oY}97-crS+KcxMA$i=Xf?I835j z<+x`-Txs)b*I5wx@Zs^!1{!ViTFbk?075u`{%OmnR+(Rso4AJ&IamZ zkBOXEBeHZc<9w0ii7$t_=FD;QX*qpbNuLff2WgHQW%sG;EQJGG*dGiJ#N~g4C%hb0 zo=&@OOZTzq_l0_M>mKUx$Au)ZQB-GrN}caePB7dD%#(mO9nI=|G?Q*h=eoXSEi(aqL-to@vc;4>ciP5Xx?WaSp?AXufXX@DIOI^EeFuZ?BR7e4XM*T`qA`g6mj`Zwe(GXr0!## z-BE7CpB)WRmRQmi0c=jgwmH{AlT?0NY15>O-{we(F+(DsR(-7(Yd zPj_ErI)413ODEw{^TD;yNqfA#vpY5b|HJ(B{snln);&YyMjsrhyjx)RKP&bQe0!h1 z^^G;@hH+@|KZyeynliq6ivtU}?Y~6(%sbEK;@@H|guJF*KK4pfsE=b)B713 zCsRh}Vf+d|Q|ehDJJ>@hS?0+=4-cEt;3N+jFWlz1S1*p~c>Va8xZ30OqrJ0C@BL_{ zjQUZIY|@AGQ0R75f-Is!Ic(;@Vchzh%r7_&Iy*6%~wcl=9x*6V&@& z(Y+ZxVe8y0O5MHn{{^@1-rm3At|z_<{L50{7Y!!Dd#U45dwB)7M>`K^MjGDpm+N@m z8IMoWmY)ZPx-_oi`ITv45DtghaCAS1#Oei(KRkbIx$w3N$2~qc6mPKKw(H5$$n&#P z;E;^<)346uG4bGDd(L(C=X-mBz4sn&t_gOhU%I(hb?J;C@ro_~vDv+t;iwHWZ49`m z*Bk&XoigR3j+x9Wioab>Y~%&#>$i|Y@fLKOThY<~Z}#~9m>l^N8E^m89$)3%ImRB} zuiNzMsRN1shPpB<+r6g4&7u>)O=Svh(6Kj^;_KT=eU4{=Xp&J~WF?ji&-EOyCj_%w&z|QT8j?7_C;hMkOrz3n?;XnhPvCn!6+Yh_ zBpx{yPOgW~PN>a{?l@^kZIiKZLO5~j`L2tTOTkI1t~~V0F#G15y#g)Bi75Nu3&=@V zjxC*w9F#q8!;FrX$}=X=yFKG%|+p@pFEslUJ|?z9+Zfeghaa z?ws>GzhUn)TqYS)?LD_wX>7xjre%>1FVRIkBje9LplkmroV zA?PAI?usWRXITGmeZ^-0&+*Q+y_-JjY}(%R7M*Lm!sSbJvmP`q^2;Lf>N{h#yL|3r z)3WCDdIvE5g4dd<%wafPuX>Sy$nwlB`z_D_iGqul|0wKbzb@;FbMe zxav!uY5f0LGu&@;-FM2XqdF7=p=*|V)&59NZW?)AXQvKFx9AIPwEs;uv!C`qr%loR zFl~wUE%r4dve2dd$GSrMN}h}M>2xfeX}r{>ecJcXAHCeBf1P+~PoKyoTu*4MyR_L1 zoZ|N{A+HVakt83=XZ(81WdZhg<9}Hm;9S8#M(14i!uPw+W#3l6i#dZ3IF>JgAD48^ zX+t!>=l6Egpn~ht&zr0ty<(N^r@}+uX?Na)txDJ)|uBG7cI`Vc4Klh7X#lgr5?wvUxJq~I+``0@THguNh z9S7fbad6zZ+{M8uE??|~C%y?k+|KVC{8HD0k2i;Xmv+;gxBMQus%X-bMuz11Wk~0I z@5p|w86SE^bm_YHWQV^U%}uRi$yq`>ig}qOo`x1DG&bBe-+RUx8*9Nqk9m;!#LiWL zjTgrk{XdD-bLNunGR~9j8Yl4DiQy>C{;H>ZIP!8{X}(mOWhpjqTc<9^9$VTJzK)*f z?0-2Se7Wu4Z{sUF_C?9n6g_7-`RHnhRV)h>M=fG&q$kuE!?`C0)4e~TF`BX_`gOaX z-QyTf2xhmQMJ|npq=8vF@ZH$o%gLv_G7#vLdAC2&UFLm>I@G_F-V5&T?ZRDN3Vstm1kS!!MaN2ZZtfEq ztGTo|5INPR%`9jW>ZHxdDc^p;w`C_6b+f;{F;=?DimaFJm80`)Qgo}$R%AM|>y}q7 zVnopAXseO2BcHuT<4?ZVoezGgwgSDMjZ+%1B4>ijvdm+YDYhbGU3?!`HeShm#rmoz z3pLK4{$ZDF^v1p9UZ-9~+r8>lw4FiQ(yNZ1+f%Pf$5Ri|=PAALTQ%<#Pn+($yLqQv zp@L2Lt?Ni`7{~n|-F8La@9<2yK7Z>z^L$pF^C|qZo-*Q%?}(S-jqiv@0)fa~(yO3* zTAp5IMY3JGr+vTKiZp?T(pRm>9PwJfitMM}6Ot<~O)jN>8vp-*_hY~xwy+l*bPb`_&t1UsFJ`sKw8Ikv(-wEN@h4J9i4t^g8 zzjMKB@3~t8?87qc{hm4tw2jzDqk~G{s3!FecwZY_58xr`pj%|S<`)Y}u^XF6BM!c$lY%i(LJ!vWH^h=@LQu3ZH zCGXkJfpe;tk}J*2xpo;buy2qD^DNf8wQen#)?s)2hWte7U}+}iG>-jYtop;$iU7Hs zRxRkLVy#;;y_9p=j8%_KwN{NL-=L8-pnTtyp9ec8tuw<9zHwnu^eJ<|&KulZZK3iB zQ!?2jIE$QWJ6O~I`8eVVv-|JNHc#tNJ>{Ash-Wq`ADg-fIheYh?Zt$?x=QRn2%7?HaibYyJ1b0Xe6oYHiY5#p2NJket{Uy5 zAaC0458?4@(@5;=AKtnWn5A1t=IXujgG$fS{|V+@&uRbFhr#aU1@}JUAwHyR|05r^ zY$EQxJvbniN8I!4Q71U@t#iHxn;+! zO)|om?TQJU{&txB!_r^9m|FRgUKnZT^WVA#c}G6sW!A;)wLP<9?#v-okLD6n{v%tw;ZS>CcxGXdIN$?>w%mAHJ&z-WD|oTk*h0`%vf=k8c^o8Hk#dIlVMC%P-=I7zA6x@l2En;^^o7wyFh^46-7mmiy zFrqVoy$Tr0&~3;|=H#^3y>RNw2bik6)&aZii$R8;jtmb8r^nE5E0|B?XN03mXotNK zktM_i(%!D}F0$B<7uRmgc`5UNa9jqjmcfH@_AjXH zeDNH<$P?wcv*5XW)pMb-wd~d4k)v*Y!uGL6oKNM%;@k40W~wvZjMF2{jJFFs9KbWT z)8}UGw@Jb6LdLm4KZ-Z^+|RgCa#i>^#(u^N_%2mm{_8u^XS&-IwC(y!yYjZ6)Av}1 zPyMda_uVjeZg>JYGLOC;t3M~D>Cbg1&>w^EL|5^D7&(*ATy0f^yW4Iq{~NhRc|17@mHMDnQOerEw`Nd8Qb>0%koLAnWC#@ z6UVzS&<8n`6)S!txk2j-TZ)bS3)!>juUGS=_5^VbbIrAB;8Z-}%Zyt;tYyq&wVx&@ zYCn+&Q|UjF6MxVB4=J1Lo`r*4FVqNquN`U z9b0DmnX_USBX_b-1vsfc6C?dZr zATE_jiqB$KRH`I!WLA{I>wUD;#ow$RF`+=gT|95t8Vyt_j$wC-+kwtJP&-z z|NQB7L1LIQ$k)(4543VLNS_>9nh_}_-ng{C(YOM74~O3VeY7U|Xtc+08|MtX$Qk5Y zA45-i82H%FQ%8T$H^_CO^Hqk$-Z(&R*GI`wdV66XMdrflc=v6KlRJCws92i$pklm5=R!2186;QtUbs?i+o#wq8KPo@;! zJi4MQ-*m@<8RVs4zeO+*xeOkVAAA}5!(sT;UKc~&YTmJo!;Y;&yWX5*LKo@v-khWT zwf^>s;BU1z8C><&rmOq+m`8@Vd?#7|laXc(F^$7kv(GvU{!fSBT=qjgMVW6>rnSQv z!}Y5&3V+J8TX@#Qo=S~tJ=0m_iuZ8VgmYq_16NwNn?sx4vs|9ZKJxBQ;a+3Qz6VG7 z;7B%+={*BZ>G6L-cNN~Mm}e$aU%K^W$<2K5R?nC%ATEA7=Q-xFS8ckby_WC*_E#IY zo0P%34E8-EUoQamL(rA6`)=s~qwziV(WUZV-2h|WZL56qR6k^=OZy_uu#}wq72h3j zX)E|8GaHfR%F}Nd2W%SefRB$w<9z%$xs-2(FS_X?+T;7DZ$3EMH20f~N6(J-%bJb_ zKdpCczn*Kl6R#_+5DLFm0=zTFwL1{=Rhb_X>1S3!YaWLy6>;TafMP`dH-~=arK2wocL= z$6wxLWIYEwO@DXBLYz6D_4#29JgabJyPhfbKh4@*Av{uOD1VGEcTnp7+AwW5d;4pV zO`GLo?-*61VyEbMpj{X*>{_2qjZH$-Qfneagv-8M6~sBi&g zD`;y0`(btUx7j|{D7c$;+sB3r)V6#bncCY++abH{u01TCZwJ{Nv-{RmXxnMhp{;PZ zunhgf%T2!$Iv)i$%uTY3-t^m?3z6^A(T_aV(=Pfl_>EKlvFxJ1@m=a1Igva+WysGs za-y_vSB#!Ful!TT>9YW`LE`jqmGQAhV)j6CwUq4iiRH(c0~razjmoVmjBKfMNg zW9dgJxhF(R;&lo;z(o(;@P8>EFg)?9SeyK57@*kxhj?HT*RJ9B^9xZA&fnfJ$uyX}+4w)NI9 zRJR|dN*5;0jqkX1_xAqpwvCq+dmlNn9vv=~UjM)H-`v-ceyvG;6HR)9|8~}F>^Tbf z7X!bae{K82FeBq0?-^?{igVrdofl(qpdLMjScOpzh!Mm{TxF- z6*KZT?vK@%p(oInOt&w`qLXRYlNFl{ok+G-)4}f(T^GE9KHc4Z65T!<-)Q1!bK!efZrbbk1VV=fD^8j6aWy>Dlzf;X?UBwI}=o z?0L@a*}3GiO)YES_vI(~?0J0Dvvl4#@yaJ`&okGJr)|O8!=4{wL|z`o`Fv?$3uDiJ zll{XM`qDehO}FvC)1EIj9D9CsfO8vr=3w;pqT^iSpAXk!c)FbB4; zHVSpdt?bcF!--=?$3zdY&jOX5uo$_dxl;Ck&RccnmXFYnL;Lm+pBO^d3>ikk&VPmf zunE082fIMuv;^`y*2wqKzvD&SOIKGee+xfIh&c)$^e%mS2wNono)OiWM(85+oC)Nf z9J1!o5Pc|U8)UEZoNkWqc%Qw4_PR%y97X4l8;5ld&eW<7;cwW%7?h4wK)E&jI$kt6 zFIl#ZNn4`9>Jb%lr?EaF{h}!_ta>kP%GOby%NfMGhk+CQX^ZypOWtX1!Y|M3-1)T@ z+MRyL@&Bim@#Zv@sjzuiwC|oju`6$)=yQHVdX?<9HgXLho1U>M&x*7!R?g9>%7yuW zRXNV}`v=H58gz1wj$#ePrP)Hh&&AIVtjYtHSO?EYHdUebuc6OXpG8}ypBvm?3*Bws z3^{#-59x+#cN^cUy!IODs`E8o!T#&;HRdpMg!&G9K#F@c#POrv7RD z&!+Z?WHhLq83V1$GxwKtWGuUz_V1aRSZpLZb{bm)%kG<+yji(d@b#@eIokG{q0!`% z+&|B6skt@ze4oPP%3;wwvtPr(l_NQ4XH@hr-zWFc>Etm2Z@=e%DK=pn?Q3iVc>W$V z5|2nGKo5LSP7b8Q^NaB<72#`f_HkD|fP5ogi^g^w+4MBLXHcg}9aZFf5?_iZvKdd} zi~3Z)C@~{{W-PVRw&H^ot7Y>b_bVve~#Q!DLRuU8{{{O(&r?+*23Il6a^zsuQO_l)vw#@ zrPcy$rT#`@^!#JU4aOOqZ#9(sI+7jmuhA#mVB8mMF#3g} z>#>o%zO{{T`ZgJOE5J(=GD|+sJr+ka~;JFNb*E}KF(#-rvpBud#<}={4_6fyN&v&j|!)FqEfuACB8M#)@ zHV5wfJL~SP_c-S%a%e=g&dzJ#9KqR77?EF88oQR>dFNYyo!&3{VF_#e6Z%B|c1GW5 zd?2~nf`MI{TjG}h1NR4>yf`|G97!Kc$JTc13sKk2oJ&|hedWv}A51qA2PWf}VGf$$ z#-&_shSrvIR$~EYHL|~Br_TI3`~~KxOYmt`{#!?``NyfPXH-n9!cS8l9Ny6it}9r_ zuBH8Z!H3Pe|4259_qy=Wm9N|6otLlM03W|%yc{Ga_j2@}@LvwfvIvSB%8IwBu<=61QL2$_#5?L+Ov%@_fGRMs6JlnF&ceZ7cyvf@4_5yi_ zD}3id3jaC4d@C@^SF~qBII$TXX*(yJSSa2(Kb&|O7>+gGq-z@Z`j=#zH9H^eIi73V zlJmxY&aCl|Y4~zdXp6Mw?X`r7yu)s>*AX-Z=+_pF8R^E}Sg53rdM4kjWUgd^&P>*SZ!boj zz6FOKzbxk)oeilnn)}}d$DfeH&0B+1?(x3R+lxcfnvFev?J-FnENICZ$|T`Va=n5W z2FdtA$kU&oV+NR4i^)M3rGLp_2I~m@k@W+xdj_&@bQ0_TgINC$MwF9N`6M+*qc0`) zj5P{}G4@*nGv2L$f4>Gz@aaC|UsK4#A0`=@B!71NypqD1K!~4MgKwi_h$o1rtX=hH3k7qtd7JV=Y*%G@5`9ZD}%Er&HdXzkFJIgpr z9Az?*L#$YQQHHIPDJOM&t*b8h%EB5P|)ke*wz#G4pSeRAL z`iP+%`g7UyGX(!R{ZU<2BRXC@%(H`w!ilX{hogs;dy2Eq;>e&ReMpWC6vjD6dOZ3r zF(tcZvky)2YX#Z(%T}Aa!mKsrB75yK^j4dZI-b3C2YDz?cJfpqkLSg)YwS6SGxs!a z*Q2990$UIw3DicT_z^`8ve4|@(T~oX=bk;v5DG8m#$@ zTtL_ywA=0FE2SU^{62>b~zeW6bmat>#`iEz!k4Qp~}t#O8Q02ltX! zM?CV6vG}&*u5TNAtAz50d1kolL2;h_*sEjkbk*@uaqby*?8`0Gp|kTeFQ>xt?CA*J)UA3YpL7-di}3BTmX-Ci`%ZJlRGw+g zS7q${1-@~<1Dxx*x4u#bZETwYe)enL2q)4^8~RWtTO0^T#d4};2%epJC3xsU1R9^SB*$*pyb`AGmOS_;k#kjqTO=k z=DX{pd%}fD^tCCBr4iupQ~IlQWO)Be^)DM+>pw0o%o~23}_#-#m9s^yB|BYFdEb;v4b1^uCtBr}p|=dtTQ%n&@nov+3-V z(;Bn^9ezt4@c=SWIh-kn+@HIR{x`zo^7;MeP<(LwpU4=1*X(x<+@H&RE9;>z?J)`! z2U0`sTj4lP>>oDag9+%MGv!XS&!gNv|D&_dQ@ZuJPO&e*xdnLtK`dxXAYu12l09aCF}RZNRs&D_QuRew41US}$fN8T&GQ}VHdyovJRYaSv0 zZX7*NIrHlG1Uu(UXIvC?L{~ZUUH8l%R~la-VgeHept zq5s@D;IRJdw6C##Yx=S7mG#EF;H^j3)qGVOd}ONTtNP%`jzYIioujx89LP6OO1<6n ze21^4w^h<-M{moC^?h9WY|pvs$W5GW8KJUP*&0mp|&6id?`K}&5z26EQk`-`>Et(j8me_#@(P>^Vg^ z^yZa3cp+8Z{Ch9`t*6ef@a`yAdtL*^YW({0C+XgUQ}utDXZ|(2B4TKqy&yTUb(=NU zXUD$a&V~9emEVP*8Pp+N-`@vpRl5(J@Dt;b3jYOslMW}}0PcL=z2SrZTHx0+!7sQa zE2J}h@ZZS|vQ1uf%WAKZ%6R8{de;#=KPG(m5>w%mZI}+eD`_j2dHM+-d=Ic5=HYd# z<`tiuc$ato#hCYCzS>?p%Z~jCey*X8_55T5dTZWRZn$t3cn#$mj!zFXXk+@=UyYm!nd@3Rk&l*Dxy8v<_Wk6-0^{}ecKk<-HcCB4w} zvEJHrY5%a>=0())=abe9r(OeI(_EnOVj0d}m~vlCBy&Nx_%g-+?*93)W11g@Pu1^Z zgqXjFFFPd-l)9qq`URgIZ@SJVHC(7}1{SKJFNi{47q>&tvi07drWoz5n#f zPlJUnF3;;M(|aDjg3dC%=fBIPjB?rS!OxSL$L}`}Np5t-LwGcANQ09+cv86iXY=?O zoiO&^k5BNA3+L9l-uv-4%BUZIW-QotFmMYV|NiuqF0CX#e~%ud_xk41c87-fJpT>P z)6FXy6N&|j2eh`*eO!1n`O@Ek;iPb4H|ytzfVcjnl8zU~7`vkI@o?nkWNbm@018W9 ztzK#8B^ZvbE;~+nOM9z#7GqcI;|Jjd(c>d~yh9gPX3nBt$7)Z$=XCA0Fa~6smyYzu ziFJ=PIM1ea_h;T*p}tnqSNTc(yzjv^7Pu5ga3%1bPMHc1=X|%ujg7TwK$%M@BfrOO zZW-Yz7rkdF`0-#Do;tv50$knTE7wZ@dMl?sN;cGT_43sn-Q>{wDaPfsswYMEi4GQc z@sB&d47&jx&x+j*eCGpSI(`)1hUOMUZyUq8 zC-BwV;7fFluX6b6lj1JE!Z&BrS^d{IN|!U~rA_Je80k=|TR85ZPOpB^LThlMh3XfM z{q=v(#&LG+D*6ye!LeX}4IfXc4E_=4UHfILhr@^sTkm?x8ke3^Xv@!s--|nRK9A>Z zdEMzO7!1YX*>DtfY$AW2Hy1q83mi8c2afApIC6Y&$ktuu!ZDcV+fv|=TsgO2N?a4? z3W!EmQD51&XPgk9CHAG^Gne=M^XM1dI*-NQf8OWd?S1IBJ`KF<`#5y+=+lTzDtwA| z&oZthqkhe|ck^wJn4$lrSkD1AuG7bQ_JK}*yOFW$+KpMU&mI$eUEh6b-)G07DLg8= zJe+jPrr0HaeXOg$o~(=8^<>31P)}-~RhN;=dIh|;{NC=lNIqNNeU5#bdX#Ts2kS@5 zH}Ns+@yb>4DgKq^f$q6T{+{yffAj6Ja*_NebnPt{$tlPjFBi!lX{!<5*@k}P%{jkH zY3nJqb(~xzp${oTK9DlAyDV*Ux&(i1tcLMvAUf$ixJK5>!zPpKc^0hm>o*i4qzL{~{zwWjxI^D-J z>G3~vpLy}Po$;3VckiN%Xndth*to z%;u`G_%Y+K4tQ$$ZQ^(2ru647v~SPZHNEZj2a~Kb9t!qh-7y0lDu^v@#ai#{8{L!a zN16pInp_+xOeQm<$*YXS&CB86)n>vZ2CX$1h-7WYXSq6kUX<40fXLR__^`)?3qBx5 zTd`%6hpZjhlH>Rd3*=W|T`oGbD+uOe|PJ0!6U-2iMD;yxlH)lfDIq_}z#I3PE zuF-gs7^0imH^06p(LTXspTvMjL0z9nQ{Uo-+&+PZroLgWMniL;s3FfZ8+M$B-}Slx zYiu(b_6)kHAuuRJtcOJ`$i)qNzR)x+IJ#+CU{HUaWfBK6w;@=4RYS1YmxHWlIaO)@Yxh@`9 zSf@6St%{4xxvUFuBPABTmxL^aWxuB@8X=QVxw^7XJ&Ne?m%=~(Ay8Ew{R{Mj*5v@@N1jYE4Kuys6oH|6N#`8=0>ZtM8Ib-7NRZ*`wY zp6t1j{&vpVgFg%>&Y~WbpWt5g?mJrJr|k7dlN;op)BDu1(4`{|h#a2f_&rrlW9J>d zlaELFJT-2g=RQsx2`JHVy4UPR4nX-oOs`!N} zk{xRpx8u>Fy|KI0?09i>0r94vGoz#L3`9fVJ#=R(L~Dx! zS!+ItHD}@=Rt%0F#dn`?IP2=A>`Sv~r|Km3Z?VTpaw8fj=x86-@nSL=h$eqyM3d}& zv}h+z3}@>ZtdS@$Lz?*9F#b!$=gQWoOC~?k8Gq`3C39dt^-5PLONqmMRqfNR*0mR6 z%i&Mmnm-V~D(m3n*VrOFAJp1AIU|+>qt11!L%xfq%zZT~m+zK~^X>p;uL5UkOLKv( zpSU{5?(aYY%4=<3K62{b9e>M=wz58SE;$}_RUT!{XNnnXTirah^}axZ@`1Fi-p>DE zL;MV2nPVj4S2_42)@+{EgsKKOupWTM@s)j}>jwZUYY16c+~b#*pVG?mU>_QL4%mN{ zLp^+Jx!-Ed;UM47v*FK*&9dR|vQ0M&ewFvbe?{M@)+S8B&-HZRw}3wmjr{wgDu~$_ zN^YNU$e4I^#g*{F74XF6oQHlH=R-QN2QBQ6wH8ixV|Kf*g*0pLZ0%%w9pvWI%o@9_ zIi>fqtrP6BWt2S=c$)a$FwTlx#C^VZA3ig33HP#R?f123My7F}>)lt3k4)uWxnBM4 z)t_;p&5zrXUm&JEGumFMokE&bw$e*+;AR{Jzn#v}2cnMn6Kf zox}b8e5+^T1KI1|dF{5J#pT)0sUDYS#jDb%?0VqUDD`OF<4)=s?mcH*{Dav-egu{C z_(i-Wp7&t*1AL;j*c)9KG@kzI|ITZZG1d7UZNABO8dJ|8pBC_Ks($XR7vwetA|~VO z8uW!Sd=&XX)*SIOOXtviJ@eD`-w)es`b*J|Q`hvDkZY#m_VSLZz6NWHhwSz0S;W$K z>r8E&`|98Oybs%1`x`1&&3w0&7{>#|_cG6R%Og?A+7x4c?xy16!Zn+V3(Hx*k-S^8 zIZ#-||9arC*YYVxJss_;1DGq2H#NkPlrk#QpFjB!-;3`v`6kzm*QqKF z6#j(rT4xd6#0MrnM`s@zyU*r@tk_xb!teOr@B6Xlk;AEvaY5+H@YeC6mXR}`utv3* zz8~%RXj}1?}V3_nw`ddw_Uv__G|I@tz6qwj5|W zDy9ddklz0(+b-pd%hsAbD$05$ovwO#HBqssi_~m>3Zer9~W8~Q` zi!b4Z{hhg|k-Fdz8P4gkB`xuAs%2nl%?*e zV15~Cf0q**PPzZ4&2;%+BnvJ$-5Gn0$dZGMImy{shLM{@vAkO-tqgU{T;qHS5Fva<2NhzI&%1SU`g-iCf{5JGQlz! zTTU#ra3We*8Af6leuv4(3&{cwe*>;AUJ6ownJb&h7=zwkrY8K@+XB|oB6R=znw%JJ z>f4>JjKMr9onQ~LVMw#17g&sK*<|SUHr?WyV;S3J$dGczbw07nA(OMZ`!L4wlS_Bt z{j=BxiA9|0;OK3WB0pr_TB$mKkvi=0Uy3{|LmoC!=Suk9PfO*&=#=-h=R`K~z3f}3 zzH=fQxlcFtHfw%l?1_*3`1#Qz!dG_8kE83Lg$r*)@GjQ9heP05z`d=HLH}>N`$n(f zSqS=B2J>4{KYRZmIW#OAF5~l@-y3l9HH7;_tjgpR`F(UAx%#5{Ds`Tso~_z1R)M}^ z(;6CAp_hms1zS1iRSW0kW7xl28Sl)^Jl>{%*Z5sc|H>$%n1z#G<3G67JVhKRcd)Qt{@MDvaUM1tc|5?U-M7;7B46acgV(Nh|3mHi+7gb32#>yJ1G#tX z+yTthJ#F1j@V|B7e>(urA7Y-C{84_{J%fyBE3)L8t;r2-JTu7c*gA-utjxFZvCIMZ zty=HPXo%ydieIbyF!zCmHvCs@tFZ-3IIDZR5&0APfMlEadN1QwImEDI+2`0iReNmO znXlJC|0-mH-}dTQa2fX3rJeSdqgM@!&9vi&hR5zrNcKuj-qL+fB64FjaxzW~w`Hch z1E)LCxg}F1KW_!6^ewWNZ{DZxYe$(`sXF^Y#=K-|7Hc?%230&LeNQ>iI74-R1>?FR zu%N@yd(Hj$BV4^Vu%9y@y6NBt&=vhYXjfJ~V;r&L#h)oArlu{UV(zEp@$klwbV|`& zZE4NzVcHe!@3Jm(5S(f)S2VAoFD3M)SJ>_N0{yViVxgGp23BWJxNm@$on@i2C zozsbTuBe>z=_20E3n`O3IIDUQ?>+q%9A@>l zcQt1~I57#GHLI309f51KnYpk1{I!qvr@gNbXMMeyQ~d_#=^i-$TaS(o=Ion&?z%_k z16L7p<&Qj{h0jMkb3FbDY5%&Nqo_CjNxxrn&B>vinhUbgdlN2?ZlOJ`Z5!mct+jbH zft_gX$NzQAHjwKNxIFoQ$CF=%uCBh-uVMQ95HN`5XW2B*w)yoD z%BzfE6HmWnmmvoVWe%~<>+rO*NA*;DT`VW|JNToT@7D7Be@de<#~&wMH0o`f^g^SP zpb`1fvY8_k!>+Noh4wmWWRFGB$neo9eH%877<-$EzxDQ+d^o~Kqm*}l;++i?rIB4GJN6`HQfai;Ui-<3-3*P!@Lit1O*xlVdQ}y=s5h?d{g)mN z*msEi@oWYBQo-2HMc&%wnGa^N$8Hn40B5ZxZsFRQ=dF{6$>o(IXFmjfl}#gSAJE<$ zKh8FRmvnl8qjxY4()e*}X8TUZo#$H)ZuK9(uAN^&b5(zL9rGc4x_TF1?Brtj zKJkk3S&bgZ*aY_0Ao-M6lQWsMu(CjStNd+i$j4*nLvV8@C=Yagd0EFobQ0NeU%<}S z950`)Uw5u^b!XIP69z1!DD9*z z0sFL>k$6YNK7Hi!wB@2!1gW)F5~!LmAQwdw3g-Q;ea=ZHlL-cF-}n7p{y3kRv(LV) zz4qE`uf6tKYj3AMC-WdOkKk6FFOX*lbw!h*40%BFeWViy1o(tMl{21J`U~u7gx-yJ z_6yilWh>dYBRoYWJaM$+Ait6=!GX0ODKcUC1+e1#Izen|!u!(>D}Ed1e&t!_zCOWjLU)2|(ap%ZWyL4A=#KhZ zmOhHkjB(5cCl2C@ea?J{Tw5di7~Y3hk&go6N3_4vtal3ahKQjO-e2Jn`(&RMLEl^2Xt&*V_2&mW!yj2Ld<<48I8uUx zX!ngM?OOABLB7FLp2hA*>XUOKna{<{tqP&<3_{&`l5OJ)51ONA%0*VAq(;yc8S4 zdB_uIt5@WaX!ULfxA2a$+ArD^jkfDUtUa`vJG4EXwjJU8h`vglyWms8yHvh%V$ksJ z;{7@qcbRvjp^H*Bi!;o$ud-QmhvloMPv>H@_}1gb{-~5O9ofa03S9q8o4^+y7y9aq zJK6i3k4@lO)rZvJZ2EA9cubM=Hgf*do9orN&YynWoIkgH7=8XsWL$-x4gt{ygD6bfA0l{;DqEX3bV-gTEG-86)8ZzfrO}_lH6=HKfa27ks+mJ3{B3 z;>w%ruEyT<55UvuyU82m`(WUZ{c`v?N_@yd=4ByuHzG?~J{r!i@X=U=9xl@Fq0DSc zd>lp-e?j_lHoN&+WC6AQm_82HI;BI$HL`z-kTYY!hYR`J!jjWv?-DFEHPvWbm`Yd|w32`^>fZQ9OQV3 zSu_lrrc(4~g15zTXA5a^1{TNvcx02rYxv8Jxfu(c4|3-L2ur~D7cah9(69w)Eh z*p%IUg^C`NgV)l((|kkr4XS#PgR|V|d|1<33+(6w<2bvD=j19O`8CnAfQ#u)qnn+(hTQ0hFbY&3M=PE|L4iqP55DECty)t{oV zKD|)(pN`;PWjwZtjy8t2IY+#&N8k)~$atngi#qfpI>~9!VR4K$L1IC<;SCMwEgkp} z)yHcs?pST3oVhvBo2NmyPY~Cy5*t3=mEOE^kk(Sg`A4MQT$O<~E6Fc}*T$~o(ij!o=MRh&bK4;ZH%do&k)+yUxz_QGb;v>lN1t#BBnZ^bruK&MQ2 zx=YL0r^efr>MI8Kopcu(^a5QeIacEU&VUS8HF2YqpZN71pa z7r5yk`iJB7>BiZK<&&%0s2}eeL|v3oXD5Z^*O-UujKwatlE!^P#{9PIr8=H*sJ@E5 z6m#d|?$*`Cd5P4KMjaY`XbhJFpa;taMU4e#J2Lh%-U6%DCyyC}1ijE(mM*QoRUf)k zL$2tkH~q@`(CJg(F-w0MOMQ)uIcw@2J#o{cR1jCU|=aSYM*A;?FPvn{Sz4H44^tU+=qt;RHIFO7e+ZtP>+o z`V!4oYN_f^=Y3H$pYH|KD=^927g$Q^Q;h{5_)hg;6Q^V%!{=MtuVw7R+rL5Khjg7d#PFpQ9p`oCnEZ|O6%lY4ytoT{ zWpL?+roJks`O6Ja=oGzcS9JOmXHmjWHvv!a*=VHR9k?4BxdXd6MI-$7>xo_l8hLrE zzFW@4Gr`~I;J+0*Ni*prErL!y@wKjzyAImuq~RWePOv$T+$$gSNng50C+|{kJDsEl zPrlc>M!^S-ybP>DI~CB$N#K;TfHpb_YyXebA+&ubxEKm9hC~@3;m3vNxHq%??(BQB zogYZMXW!@Ksd0igMC{G{;3yie|0S_@Tv7IBkElMT1gFwRnfu+@o2>^v*_+iU?A}{1 z(3Ct>TanQ$>+!n5+HQg8Rp4yoZo)&O@&bn^76djlBm}*B{LX2*!yEU;&Yh03?T;{>8RVWbJ2zFx4nE{OnxVMXYpij*}ij8*{;Wy zlV8$iN?P6aomFM8KX$J9R9WhC)NOyvL4A(0T{|^klrslwKYBLY2wrU9M{r{YH~9kx zcf!rOZ5gj9yd-P8%h`+F{{IrZM9L;@d>L6QLr)1gk=unn9jtMm!3!j=(i6lQX<*G2 z`Zx#=*|-XuRnBEJ-WzxqJCQ|Z3>PUY`$;Q*fV`6LcAFM3Vg+`TX{_}=NzJ*G;B5HP z8S3gbobb3dI1k)ozzM$-*+#;Nr#C3w@rhVIjn&Bg$W$VHKIYPg(c<-M4^(vCIWSN;Y4zSSn{STeHC%}V!> ztm50qy1fh7?*;aIx`6*A@XNV#GUu0KbEo=|Z}dYm^)ikX*vptcP(`K;JM?dcT-=t9 z{=kxLe~BC-chN=mN#GWK94>1hySJ5FMLF5C%?4Jrj%1nml)fQ<{Vt+!tLa-o7yXiP zk$$BWX-Xd1XZDFXH*fEo4+aE|V%7y~uPC-L*e<=eKJaURFwQGx=rgHv{Wl+S>f1(!igvh5wu}Fl;;Ua{}Jl)Q2 zeGoZLWR5Gq`KRzE%RXfyefX%4zWPHRWwWi@yvkh9-X!$wHb#ExHd1v50quIWzqqce^_z+OBqWXIPcmvR;OCk-nu ze#J|GMEA&#z#H+pi@Zu$t zR`9JP^L77Xz72OiuxCc*E`R($x!)`<7&j2xOYSTzr=A@826|B9xez$AqC45^>zADykVm1YKl)d>_+z>4g*|+?b z)_L)h^3W5z1EY$In?KPLcdno=srL=Nm-kuz*KK)I)f@lcyLLWJy|3%NcfMKo#?D^l zZ$0KC~eI1Xo)>Z=EMN5k0hps{{1q63yXVv8d$krSyG8<{N%s43>bG zQgFGD?{UAfqo19>uNc`QySR8#3Hco0B~EZ+ev7s*lsS;%avQ#e^wYt*=VnaPX+MoS z;fY!JsMywvT*$edZzFSgzb!rFf}Shyy$gPFx3XW%#QxXd8>zv+z*9~_ySJmuTn#PV zk(C#Cf%%xBCx>KS1(>6m*b=#VC40+HJg}#cxm0XR@z%zPUa0dvKzPUw>}+!N^pFm0 zcTn%+)O#oO?g^`RM_9eo1Y^~&55{weS`+9w8y5$>grZFWpDU)=Z|)iN&~ z`dz9|d0~C}6@3z0g#OHzuJD)^hDTXb9C}))p8lqT$Cu$Tn}6hP28X*pgrA%a!_Q0f zLF{!i$lnn^X$n6zz`gw|@DpDD8)p4?f~P0IllY|=^`}SFpYGiXo+_!o(Uzw0FL=6? z_x(v-+p1-W4;K1s(dCF9W)r%n$%7~RghsaKbBD!BZCPdyY?77lNxQGe$Lta1o(qY~ zKap75+k2R?kkO+~gJ*BQ!YG@^8Q7n})4M0;H}AXE-Mqbx$M;jZqAiTG@a(Phs~(%X zxsS-c3VU|*KC|n1Zvela!;6b;X`zGA%xd~4{`r3;eTNxqnK~4I+RWX)0=vXrF66(B z|LEV}mOWym@3eKt#OL<@LAj$%>D@e&;jwA>#9&hZeYnhXoJcrxBH$dV-jjoSDI@dn zF13!aFIDHN$-y^Ce}Hkglsi_#_fQTwL$GP%GSI~pfje0rv$0P|hUPw{&nf6XE8B3^ zu3uyBwCTz$om0#soLzE9wdiuf*YkDYM&`4X@9*!+TGYNLYl2Q>UUpnRlY_@r8hVDc zz#a~>lC2ZEj$JhkeM#W~&QjUC%U!dT;G*y{<`6n@(GME?=4-n2NtvYdE$o|J#Ee6B zwP1aHWmpUn^V=Uy81EDq>4UUeOS`g1&II3*X0=xtffpUVCwLi1dqOj*X8ytjx^JvG zZWau6W;_2zJMrX;l;at7%J#oXEbUPsyy{7P9z1a4(M z8f!>$@PE+j2ruddEdSM;{k>hzI-Q(f#c?*odFM8puNZm4;c9GdjMXMo_HaxToHc6w zCQc*0(XMS0nVNG`pAR1Z@y*J%xh4p0WfP0pfi7L>OK8hM4EEx_)Z=hWsC2p}NFC8= zuylgx38))=K_j~QVq(2I&Ro5xi5T&6_Ar>a*gzkIcFx>46If?();67Zx^^Fm2If!4 zG+D3F>Aa==dxq%Sv7x`*SA-3H4*gZ>$INuk<-T>K4<@~~;)3Sd!TRnUIx#x9qowv@ z?qk{p{a!p3e`)+O8ZP%u7Z`tI4>Xm=8uv0?tj?OB%`BYat3RmuP9>`I%eNd2j;KBy zoNv69>2ku;G~ZG3im#J`Ps{a{0f&Mudx%f=wko|a*Oy0ZUzPsK5MK*rga-<~bn@Jl zV9Y7k=aF}iw9|9mN$U=hl;Hjqt!sV_zGOTjY5iTk34VVpd=Y#^TZ_}VZ|%=cp$A!K zToXm7JWX>W+jKm`n%Jw~1UaXQr0YB8=z;CTyv}y^o3P#Knpg{MIraOWKg9a6XU#!mwD!#GMY; zgjst0@*|W_IDKUMYsi3y?7dW70%r~|ica4JU0P>tUBM`N^c}$2WbZejD#kUj$>x}- zG4{iNHN>4Yac4r?MLuleE?n{FkTXR2?%+3Cou$Z0F1|;T!6d(fy^7eOS@Zuh#)vV0 zR?E&3`dyXpnW|zu-^2K;^u1@HkSt@io`M zyGzU1r~aGgY;tdT{98awwe<5u$J;q7Y`eIx`ovsPp$ zCQZnQb)VP_{>3(aQKB|4Ue|Y*>iY3vL#qZvmMJ6Ov>=nbWP|nF2 zPsKYWmY4XdM&`{RFZBDi_<&-s5@_Ha62*^`&H8{VjXW|g4)Qft*aHplK9wh|&0ge@ z@h?Fy6n=NQ+|w>RxumsqoUwK!2mkyJ&7kFJgACd%yiD=H&Am(-pLelAoAsJ+3%v1F zEB#$F|K`gSPk+%$|LH|0EweU3%TJR&3ZMR6%%#)(zl8s?K5CurYij3}nWx%$b_rqPv-qN4|mR$mH!VhRkS*3qw+Fh#U~r*?V0dIPhZ0q zuW}gUI9OA50maAyqF<};y?JUOeszt5WzQn}!D+O24eRLD*zy?TqikR~>#AhWa65cy z9=~FbusuRf68XF6Oslz1Am$4A_?SK!>*Qo}?X#P@D_`c0Uxm!24|SNiYNcy2t{4`hRPSM z$-e6$>Ky@IrEiuU-j1V20UEJZsuHw!ygzIOVLQgB>Ab(#Tpm3?3fB zo`n1D(YxIDeEjLg?IQCv!Sga1i{;R7c>B&UJI+k_n&kr`XELt>iyF7V+9sh94SjDz zh35PCrEvbn83Fz=*}}`i#`gpA7~^F4C47HGSfTRzq@vB`XE z*e~2I-r`5zj=#C)#ICD$pBTRIzzObq**Wt8NA;y=IguKw?+euQ`Q zeLJ5ft&ab7=mDRWUwK#mcT_*ayXXi1MB1DDe-l06pX68G)&CvUZ}HyS)_dnM(vI@~ zsGh`sXLXaj+x6#1X&!LY8`|p~j2oe$%S_t2oc~YppjRYKC#`odVW>^%0mFTAB5`zx zv6xBS7Cx}gTvd`e+!u+bU5r)9x}I(6(}BeW&YoniEcjL>xMjDT9^O+keju=S+f_s zKlJ8Iesjt+-wYmy_;Mi^X?rtgtNPBQPsnb%?@_(DaY{Z<=KEQc%bYE3XtXhmSSn5} z&O3uAQ+NL^pC{9iHOTgNZB%A<)*#t)i)@n>>;7EWGhZ z$%;lsG4`WmOk#NKJh40$-DE>IMLm-y4r5H^KHR&*=%xqxgm!xJo5a(Xrw>mu&%7H; zmrnR@>C&HmFk$4T4-}?U6CH~8vNW5wPWg}PL`$XdE55|q0uypH^ z|GI2V;?MU_8M*P^HHjNP7(X(9?`4Te`^F_cw0BD4=zZf8Q!Y31rr!62`abdU>clUo z_p}iG4Zy2mnEj`pD=R#{>w*ZlK%1D@rgpaMaaSVdj}`l_l=?6 zp@~EHjZG99+OqfJMCoTkbNU1)^c<%-z0-NFzR3N`M`!uGta`G#H$(2e3*|-xaheqW)AetbDy5uBPQ_#^gp&o%+1U43kS(r zld})L20nK$DVt24<31QY@S?8mB3}q zzr5ykBQsa6o3HiD93*p5yIX#*7-TQb9Hhznq`{qNF(ZN&Gaa4zRfEZ|kOk+B3!}SE z{A%y7X6Gy31JBy~pXz@k-&z;Ow~8Y9*538AGvCh?exi7nt1I3$DQi%sBUACMnsB~F z|MKB$7Vp{|&bz|Cb;Z9T`PREF+UjA4i9_YUo+ZKks5>EMHg`F`JtI$huAX*nF?n_N zq^%nsT2XWJLV5aKIXj=9aFyFTHuVcF~}cFMCdKA-jl7ki-42xjOc^ z#4-sWL&sg^ekG@mwtK#AE6GE4l041C#(sh4XUHZ?_^m~s;3%5n)4pBUtmKmdJ9o=F z*#~s=cMzTaIL+O%mzb-fFWHCuzOUTjlf8hPS#IYX!up>br_I|)`s2N{7i;--Bx*0F z|Exgm8zk>Hd9Pv&*oUstYNqhi1MC(TOMC~{rF7XaEUCzx4UpQOgS$&>jXdHW8~}W+W~EeZnVSLHIdHP75`de&)4ps(mWsiaDm~YBK|yQ zqw&wWpz+2q8lPs;_(qe)vxUZg(N5!ch10m_3g3$E;^#bR{QJ}&PUFNvc+TBh?5f)7 z{68XbRhH~q1g;j5uILRo7jzfSyz`^C%Viu*p5e6UoOZA5Zo3WV*Y1VXFLb`K3py8i zjz;HMq_@#IaB_cq_vYhy(mDIoj&%Nbs;^1TP0m8k+s%1Pe6CG$S7ulAExu5q3lLtO z1~30JJp2oIc%wNUjmR+8|7qBbyp*85C}WikjrTjA-(r2coA)-H)aUx%0p?CTsw_?M z?@*60{#}&Y<=pqecMHG2B`W`ZTH^Amb6?|rdd;4@FPyI1=oeU??=CFs&JPwZ^;>vw zeqr)y+FjkfcAqWD)6NSAOR4{*Pa}BrcheP*&W@zrJJJ=8?m)Z1SkgThS!d4=H&;@B z2Ob^h@2mO>+HI|{6Ccz`%&|CRj(Bu+_*dT7i};}8V`;@2GiWv)ntd0V?N{OOIfzRl zvV_pA^*skv&|{ZKF9? z!)Z3Qd$2rP!XCXV`6uf9F=6?DSTYtK!fEznNspAfI?vl@OSYU}yPK*1<*!Mz|J^+p z&z)vX+^nJg4m4YJfzQeLzQJ#=g&t=?mouTy8N@C{hNw8Z4Dq#i?8&G+mKdqJ@BZ%M z;aus}l##Lj><*7ruzb6FusmDBS=Kqxs|m{(>W|>Ds@==G+wQvaYd3}ZzXp$0F!t>p zjOR|TCT>Fb1azR+q6>Y2|C2m6H7bu)xcC#WeieUVJv~-(cA8z)NiHzwZY}k9i^np5 zI7f|yWz~w#ylz!V!&zykOIRM3`pq?|jmOgNBi(H`({{GG(q+3pr2YsVyUGz=E?DJ= zCKmwX_qze38mF_%1zo~;1@(u~>?+5#8NQru@YqN@F3YF&B=T$S6zzC6x{}qzTa=jp zVkZ}u>Yfy5*T&_bZ(E1HL*nId?t&kN5ySY0f6`X7FFGzd7WVn89M5HN2M_TE_%3$z zQcmNnZf5G{Nu{y+=#|t_a-sXt_(dg?O8aQb?7SO$ETg``)E#b5=Ae!jB(AS^?Co04 zSi`?5-HP4DJt8*N5j*Qi4C_iP>kIJ%5>R%?w8=r z)aC3)y^AJIN~WGqsZa8k;-kKWIFfDoJ$BzX@;9(2@#~W&My+#mtZ%qk=K$)%$I$mK z`K|Fuo__D7(ll*ZK6}~A?a5w&x0e5`8P> z(~`BmO>yqYIi%(EPaZaZn|;|rbdZM<-IHgKHn+cXSh6#5Y-#?LDxO9KIFh;+VmBgi zI`n?n?qykWuZdL`XC67|4;mkgo11gjpt(7z{rWmnlg8FwxN45$*lV}q%QmcuHgaeq z?t!Ga!<^b===&bMH)ob6ejU2L3;T`Z0nR*&AM7#Lv2@@VXZ@Nv*_n>G;cXGEtV{1u*Q5kv?8||7>}bcZNO+&bCRgmNWc(Vi_mzIW|CzSBnK*>W&8_2qPdV$` z0_g1>^PAXINV=rGMOp*-Weny1mnL|kOXu94v}~u$iEE(wtD*g?m;+Zb7Yc~qON=|y zCWLeCRob(reNw%hJ6Sw)#CArdCk;`yAIb*G9i!~xRC?_YUm3pfD&5t?H#XLY5wG40 zV|_BW)&H6p&Qtp%lV1=#LmrtIDxG^aWc~=B*c(G!BGF4~vA*N{cVU;Hldpksnd|awryhq+jOi_}H3wMdI7{>m8|n@`CA!SA9zGe% zY~AaxqR%-$8|riGReoC^cZ(bQC@HVfw_N0N>D&nh4%^c7i#wzpU{CLeaTABx5q#lw zEhU=WgM+Tb8#UVsBV_}J^(YVjF(0a$M1Znl^3IXDTOsqtN?`DEf9DZ=H_z9h>CUz$@aRpsnr&G*X9f*pH5KD?DD;%8xt3+I4rSlZ zJdPbOhP!1ZJK)vXDcab^iQ42FfmQA#$aTAym8*OSa~;kL+VVNcn{!Ezu}$PvWiHny z-y!r!ne6-f&UN$}*fH;w2`X<}Ir6J=#4UQ9a(A(3ZoJ4HXvWSbSML`ZHu|Uj33}YF z{kGK5-Z78+u?6q~KXj22oC~d2+WL7LvBQz?HsBLK`eJ1}ogA!0SFUiLImTCPGxF49 z2NzB^&GR|eF3^>2L3rK}^D^3f1(A3h3(VcYi$ zYvOj1H?V8c^i&gv#Bl(JwYFq$B{&qm5pD}=-H{+RVhx;I$XpazV+haBTea0vzr?RS z7rGlq+3w(L9{J9d?rx{d`Ow{J%AJI!n(xXO^G*K0#5~=XHO41$+pW+6^L>-^0c~zB zdY(+ydKmGGYl)LxKIW%>H~9rOBf+ESx;>o#x%pmc(?S*G-DuOi5}zL1wT*&b;q~bc z6u$5|>vuS<{tbQZozSR@JYCVJS|hYcDt6W))&$9)2YkY-)&X;-XOQpPf6!9e-Zx+9 zyD|J-yU=$-_`5s9cP;P2&ot~|`^V}jwTs%{hwvVahf8c2S$i6g3!C6uZv5{hCh!dG zY7fD?ZH4a1LVu~qpKFz>6Lb2jh!3_UfZ$$W(#t#wS!#bhn}PMKx% z)E=AKnxkUJDE3gEp0ek{e^TaYg~Y8ThExH3n0CePNY;qO%Y>h{+pTn!@7^DF%-3ei z@g0e6DdVw4-7oPpxcNHxNK^R0KPe0!H96ofLm$_}~=g_w)^i){0M=?+1FK^CjPcX_GT%=_;n61;5~-sBa#! zrO}^%w_9;ib0TiIfJPg_SQXXOJY^Q8XSN~fC= z%m)|8ZJo#1ah5*r5Zu#83&!=fb{N;vmdvS3s5hFQg^UgIor=rhVO^0}JQ5e-77zC< z-IN!=hYy)9S;fk)0oK&b4qpwnW3pHM2kXQ)WW28S%-QmwqgwJAF+e){fBqZv6}|?A z$EC%W^7v-zePyeIc2a{?)NRFgkF0kFFiBrppp{y16sl=-`08jSi}r$7=48_EY$e z;4VnL-=^M3{l_{RzK8e=Wy23r*b5>f5;yMH3Gt7+Ez9YHdHINguWarLvz)tSt_r^& z4$hp&WQ9HM58xBJx^Szu8%Ip86^Hswa~5i1kE%X_r=WG|FP zgST^hvOuXZcHaWlNIrbFIa)uw;~d>&#WZK$zJv}-`d>hug4agILd{8jovgnB=2&=J zIk8S}Ch<3{HCERqJpWc-)9tXKvPWR30MHnQAz6)3+^jjyH8v%BbX3TwtZyY^x# zd_v+N3*Si7Q+9p9+!dZdo5UG#`u5XyJ#7~+<{l(`|J}^rLR+f$A?Cf<6F8u=v+>tA z9*n+rG@&~X9_y6*6e8-_tl&=z3jh4V?%Y*Od#RaDU$eC@(sQ+aPx$w9oB!KD|ugyZv3}PUKpV2aMt~lvdat&qeOr@vxr0 z^Y()Qq2Xd+lQpi5?*fy-cTZOo+qSF178e4Fs^371=&ls`s0pOf(s8TJ|Sh0Cz}m0wUwa4+pw z(EgSf?ZrrZS?f_M_RbPFLDsg7jIGQ|8QWTOY-`N16`G73+jwwLYmRMA*x1IBuEsVo zrOPor{lMAzK6It{ej{ThwA4-)M*j~}-lFM9o-c5QmuYywkkM9wkI1pe_#I=2{#?za zON>4ij)6|GEzO62yiM8S%M3nTcm;D!Xd60}^=+4YyV@)>qrJ@UC4G)DFAKlHJ|N!U zn}!@}!2e$n@Ix1yGE#}@3%xpeX*Y=9;z4MO^F)PrjW{$`yLBe+qOD!BAHF@-ps!T; z;YRr36VRvdU>W0mjIrV~-JHLV(wAuScLnJ(#vesS@?7~ z{EmV6+)me*4RN?n317C#3e5_?%C3(dud~qB(|2VKd%0Jo-?yv1KpS6YJzvH**2zx7PVdQ!|h+sJb(I=JvYbv0LdQl_i9 zqRF=?bLG_iq<_U+Ieve~xi0$B&#ohmw;5Ae;1NBHoM${Vg!4MiE-G0MQcRi0`u2eN ztrpsB;(SIUJ~i`hvW$=T0m&ZU!gUevvhPX<&&L>3i*_wtrSS1$=t)C&F_kq{##1Ma zF>2FcYy<9SYn8|*=%c!f(b3)b98ZBeHTF^BbNq_7yNb{8i->YRQ}3z4X+jTC^efxx zljw&W&UW6nLdK0b*(l$L2V|}57JWYyF=n0DA7p&w)6Xd{ez&qNOPrI92Q;+?l_8T= zGS@^laDI629^yLfYF`7|Wr4-2o%Z)=<4|Od!#ZO=zh%wmBIcu<)jY4Ee+<*@iC^wJ z_*yjzT|rCIC%*e?zcl8#!=+7@a=)Tne3Jp|0iS>J1xv$J|=Qgp>4M~RMXIOVKw8sB6rwwHgexfb8# z&8**gOQ~AJ#4mFT=?$libSZm~G>N4ze7dvj)sZLnQ8@0Hrtr*uPT(X)`rG2ODzM7D z5Lk8A7tyPS)04!iS98@oOZ>-u2K`zzW0kF84V1pfepBGn*l${W`Q$$IhoqPBUDBm5 zO{57tQofOSf3Ew~zeK_jWuN+%*+z6s8v8m}bSKZKqF8Vr^#j5n4Ca)93lVhE?>SKay>Eg-t@R(w7VWoRuq^j~B@ z2YViLjH6nwDeH(R%b2&79hzKTI`9zrwXPWU0hT?0F+Vil zHS8sX-yQr?Tm2~g3&*eEM`*tcy5J71<3swnPZfe2nF|*&Mvu*LhwXRT<8*=hFg=qY zhYo-?mZv*vQn`SV>a-$Tey z4;7~Rl%@_ zDxsBZaBs<)(bkFjd!?Tp*NJeP3f>)!&GJ(d*G6_(c+83&s0A6ZTK4cTgf-TluNmf#acr*k}h$-ta17MKTKJU_Q%n# z@E5_A^(*i!AYJ-B%{;%jk#UNFZtOg6);*b5A~#twQ>DnYx+|(Lei`2~-7a)r+?9!Z zt7O<~^<{aCfz&gN`4u^Kfe*ac3v!ZoYc~m#u{kRV;^9WagW5EHDBMMtvjtTFE_@= zrPe*vF3nvPPvnO@?R&OM>a85j^=%`P)mGs-`R|<9A2JClG zpTtZS-nX&b7~{?0w3e|IS`&R$IIS(9yu??v_HUnmGY>nw|54}cWyD->r@M~w?L7Jz zj&I=+7C#4n5&L*8xRmv}ut#1?QCwcjx4~uM0!JJF-iY6^%oXeV4d(Y})EbXHEQh0?#7PACg#J>VJHwk^kIJFhlN@ zZ{q)S-N}D#*`mIy{TcjfNmc%6IPP|T7U*Hc_tha4+rv9Hloyi?}r*F}*ueCA1?bs$p-j~-NSJ;|I zcQQ`fdl0W#<`H!FF7s#|b7Q@l8_WylgN%!;za8gNT2RKN^IS>`eqfH*E4R7>Wj}BS z)?i~@KK5sRi6=}9o0h2}Pu=f+47q*TDO;uguAAL~C+ruHr`pJaz1Fy}Jlo0R&KZt}4`Mwfx@eRp_G+xUNU z@19!dxnG}uA}zyDtKivfNGYe!cNsDuX# zzb%CCM&q~A#u4(b)q9ob$P%Bh_I(v@+SZhR6%YO=kG0$@AT|g>I}5;>=s+BkxbrYu z(&AKH?htc!Nv|uLj}Y(f>0bGq^Tvf5f4p|Uemz00YrHQG_VN0sQ8pp9(l2&Valjdm z+^gy}@g8390^+#GdC<92hW+LoM~cJSWOojWPjPykDGA$Dn2-V=FV z(-TA6Nb8M$g;+W2{Rr@6*m{RPVZ8Xh__{uo{f!qtMQL zz1QMl3Wh3w6EdqO)?Md{(du04SFbBnzuf1le(iO_OHTdpxA5I)HsMAbN3r=3ePbCm z*Qtz~ZFH6Y6ftuwUbqqd{}xtfKRa#meI4KXo8R9F z`<~5rsk7<%$Nd?0?RfL^Px$veU*!)xzec5RgwL+#UFwiF(#<;lC#;S};FdZnDZ7LG zntk)sID9G=#}Q|h_>OUmm4?4VVR_k<#c3KoBjP{8d!lD5=SjqVbhp&l%~lVrk~V#3+aGj;lA-KSt3=$Sgr z>UQVBQ zZ)ot^_nzW60T{035ttLe-x2DQxi7G|z?I{Mk=_|MjPPDUo~35KCDfDHO7K_7;gRvwh+Ca!?j0!Cp9OSd(4%U8Xy{ z65H)0F?%cEX=2yVK)UQ_{;Kv@5(jBO@Lm3^cu45TD&gNo%xdh(lj#@q*f<=WIrGeI z_e_;JH!YsINWL7c->xP*zBkx{HC(LoOS{N%|KJQG1HX}c6-$}@DKHUh#3z1V;%6v& zfLiW`NMDBk4fkQoecr3t|K{5Jg|aXHnZKMk4mmyhdt0a{o3c$A;tS{Sz5vg1K@Y+k zgclUUC+An;gNolk_-SxlW`&$Rta>&tHB+1)8C_c>i+V53$)!E>9bXK(Z$vN_4K>A=LOz>5c^cd zu_@i?_gs6Fe*eJe_c+?kGW)$W48}@eY&>oD`=}m}dx$;*=3?7GuSUN$`t79Og8Q}h zln}9x=SiP053BFnVf7Wm?`4c`1MWupE%iu@*i`!1NUU}7SDc)gT8)m$C;p?~S;(Ez zj9Vj5vfkh89&D@w1I=|HKdkObb1m7i-tU+3^)S9UjBgd=>tG!luxMj7=N)s>zP+(p zV32jIh&uwUx<*B*t7(0;s!M1^>RLoyp8^Z@J=SE;4o>{;`05Lmj<22!oN|}t7W^HH z7izvH>i>eitU=b?kH5~&F@AptUL^9U+k|Zf^)079S#O{CcCh;1MZs#x>$XonmB~C~ zZld2XV#9v^%>w3cKJz$_{ly=YZsmEtO44&SscS*+guEmQ$t#y&daV~s0e&6~wKDd{sL%Z|^9FyCsR|Im&KTJUUijt;n`75fc;?Nq@!nF_`U>RLDe#Cw6IXJ#!J&AqWf-_AvBh~00~>cwc?-?9 zOQ=)$mBfaRtScWLvXgR0DRZ27@W+txg`XeL?OquRx8|8jph>T*JfnF=5%*z1w|y>J zq>mGClK~Gc-Mv#hLm+JtQy>0p5j$nlHal+nu4sgr1~aKKTXq1mFw7 z^Bx(QP(4CV^j=q_?G7<_|HGE(y+PXJ5jwsB`J^6NZh)3uHr6fZW;A$Q#@G$Z-caov zvZ1Njc6A#p^3vyt)}cvi!& zL~b0x9Q}kgkJ6^#K-v`ATuPhqz}+%CPy~2mcZtRl%>=hB(mAI*F^FqY(!hy*u=(*1RM9*7tPRtpf%x?R%IfGuPoC z#&ei)EoCjx7(a=-?!D06BK)IN_G;id57<3@n4>+lI=SN@LF?u9u&45rl}{-vtC;c# zzR`~ibyJJ`ou0 zo2PCD#-~Q>fth8%$78`K=S2#>{vF|S17E$Jc@Es40ADX?qXzhDV)Z)ik$mwPeSY(s zWdc+0&|q{8iG1hX_o2X4V89d(5A#*YMaTF}xJ(PgPiTkbsEI2VzWxGB2w0AQtHa!ctB1Qz>!H;hEi>k5 zlGfYXr(D~u+vYcOPe#ZN%$ewD%6)mKg!bk8d%!NZJeyo9>yg-)JD{}*nxopm9BMK9 zCb2nEinKs~XrYlgEb>na@|yUGNZ-Byws8mZ0-4AP1CT>(O0G!>PK3@Zd2SlALiU%f zFHJ@^It2|D+Xi?w#zKR(oZ!R>4}K4t(APZ}xFD>Ki^A#`6zed5pTr&E29MA792A z8f!bg6*(a{MnC>{=Be;cJ3ibWf}dp$=2TBN^ReSz_Vg{kSF(fLk@g~IGmbO&?a7YO zHZ~r8;Q6)USIj&Tzo}+$ed5;EHEG!3%Grj_@5j80PV@`Dc{nQ)ee22j#0-hoTH0Yq zSLYo9!}Y*m#L_X&V7r3n?OR*ZSvOYY*QX%6$X#qH!5N?}E03Ef!{+{jZ_`}yj(Prg?+pDYKYZF*+TA`Cn z=DFZfnd^DyLjoMuX zjedk*P*-qUH2Naw~4m9gTFCR@b~eF&h(dU;_uJ! zE$Gz_UTo~K9a`htPUjZ?um|+&`K;&Ju~q&?_OQ7z?&AlqA#Tj&$TP@bl|AO|aymrU zt{-|qr^6_rb zRVBkiC$Y!-jQhVTiQ&*h8=^~^OkAvuJ!X2Fkp(3kyA-*J8Zn}*wVTM zy|_Q6S8m-bWN*<)U;D7FxiB`jZpp(JVrRn`5^qHAF_JU5;jxZj?Wt1qG~e3gh@I&@ zB)Yu^;<#IfM|bXw>$CYi8M^}FuZjJN=>I-r{HoA-Het&wdQ#$oc^jXJ2^`ART24;X zS|rYj@Lb7f!v|dUWVb-$GLDCG#Wu?`r(bM(@Z?1Ft_7S6U|-Y(ZA|E??XE4c?RG4> za}sBLB@*YXg>|8^z~&op&z;TRY0cX+kY}2`-!O;ne)KN;KyRzf9@2j1KD8M<)!Nc` zrQVQSeafayEVjF!cL2MCd+lOUGmtq&r+h&hJz2DLEwm&s32ZF|TFbZCuV&ES1Jt(< z*}ECs2z){Vv+0-Q>4)!7;Wv}3D@{aq zpv!=ErlI2!y`Y>`B(fLNb$1>5zq*lqw3b(MUZ}1d@j`VDW#$v7E~d}CQw{9DL?8SZ z{r&cd))%d@JAz(vI`lq_`IO69?DVl^ez7kSz2ha+n@PRvY+BuhzFJFB>B&^iyiPpq^> zj9K+n0f9w$fQ;9H-mIP5;?JtXJQqIt(QbT9L<6CdU5H=c8}|9yNk?;QE4k^gJ+QS2w8@=<;Nlp>xyeeIM&o_1cU zH!pd3(0TJwyYNxkJYPO~A#&K+_-G&sA07O^fsejKY?|)!(OKugM}HYs|2gqd&g&xi z=#`KQFe{3`xg*-id=EjawT z{PRQB_rH(QHr2D%i!53&LZk0CVtx_dF`7Jzj9|&5B8wLHnA=enz1WgPxhHdz$wNt# zy}FV`S)1eXkVSRfcjz~IAP!y%Kiw^IDf-aea{spQ&`$?z+#QyQWo7x zJl0_%i&DQUHQg&~smN_g79Hi@G(9GYEQ$;vbFWPnMSeLQ!9#C`wngSF;Jiz0ImB*a zKJ_ky&Jv&lvF(ue1-@CQwwdy&PRv!2lSQ5k0Y@%0co2ExD&g@aT$X(L4YU3Ra5*fZ zOaW(wlcWqX*>L(K^-25?kuyt>b<=0jH_QJ6zL}?8MrZA21f3QCIfHduDV9r5t@;6|E6bR z19IQHl7EMDR)=i(1$!=$`@eu6NbEmD{!I@)a~%HIds@JdeN%#8Ak&Ckxg)8w+*?4} z+2r2ak$cyga<7~NG}=;j2}~1H9bV!hN0EI+2ZHQt(yx+zq1|htU4dzmDf6zUu6N-# zKQ(3EY0%*J=*uMJ-TJf(yiw)dh&m^kbqbGu(v)=<@+-3LV@lRd4jqJ^PEnsB>#p(N zt!(-whGf6s_rX)GCF7zqYC>j|J&){TX7h-AYuE(#3W<#C&=VFf>9gGL;Y`65``U)X z_UvIR*c;rNn&6c^!ZdI)-=2<~d*i7>=x1hywmVK}0J@xI=gg4$vXFmeTn@8u72C@r z$W=0Sx7oCft9T#IyqyIf&xMYgek^vHoL|#+4)%!fkU2Ax%lw73U*Dr_LuN{xSNOO4 zSnIr-(w(&3D^!29bzZm}T`o3>@Hg3ixjBalFDJf+x0yWuQ}%)(_PdcXb1D4H@t4+l zcOx?wN6O65fe}kQmGcz#%Z&4$e*)jHPwx@<`^AR-;mM&Ad!q|}4?ni}{Q=g9uj2U& zUTNp~ZTbiL9G3~t7rA*j_B+DwvENbr9{tA4>>U)(XKf)SPY0fF=^rFLlIO37Cm)5D znyBBEn%a@)U)+WM0enXCd;)cLkejciPT~23MgM@mbUb$QB1d%4KL}qZPbd9D1b@HG ztV`)1%rch#VYI*wue}7=+w>2L*PccH&_(_(s+?C*ilfz>CnlzXiSMXYsZmc=@`zXBjFJZj4(p>M)%4;L)d^U{NZZmmp zg~@9lSG+bgWamAc*S;e7q#n_G3%_-Or(%=eE~0P3Z)d|@IiR8#Rm^9o;<0P`CVCW=nXpYzzeQMZ$SU(r{Mnw;OT40y%p$R)6u`C z8~ktEkhAbV{QMRFgZCXkR*uU5;9G|NH945|GQ32_gS0mO#~3KxtMEK{-=~9Z0gLCI z3%@Iet`7^(pgvb0_#Jp`lX26#;CHMS0#BQa3*N_|_m(tEPayj@;dgf7cgVL<^#rsP zmEV~#Sn@1EIaFPW-!) z;&rT}?Ys`%DtwMO)a|^Ed={_kcT{-YHEpuxN%)|}>vAG^-L?di*V&`%RKsP)EDp1VO-DwI zw%3<^J~G~v%NdQbM~U&;5v_S2Cv-YW3arHmK04-P*!IxiV7KKfbF{|*`NKfAHVZQUk!Tg77Y z-UHkBp7^df@Lh4@yAl^TQ(^OcguL}RcEe&bCpK2%E1dg~c03C`#A|V!FJI0Yov-=C z7VF8W+~fIW-kxUGnAfggPvO$$-Pt!)`Ryyd%(-RdiPkk9Wa}cHtu~wQA>N(Xtpquz z9F9&dmsnxjhH3L|M-O?*9*b>EOt3hgeTiMeA0;*T?m%ta;MRgYWdrP92XUYlT<@sP zwd==4M_GvtWDa@>k>^AgF;o5T+81|vQkfs0wq>5aRen2bube3!Io7%cAA=hPYAL(c zv|dTL60vLbOuesffA6coG+^HTf)9O!@l=7zH{G{S-#Tm>ScU?oboy7i$#yRjeV83 zFZ&AquSfqVYw=;uzN=#Mngu?QuXBNK(HP1D+eTn(LQg6>9m%0k_0WUhR%qb)!`f;OG?B?UtBX1s!Kb57fAN6=Z^QzL-9N=0 zTQ&uqsJaVoBQaTW$y<=4&HE5rT)9(4{MrA`KJ6pq+UxOC7rJ=uN6;iXrSH(5jEUqQ zz?eu3ZO+k;H;t|GSM_x=7EU!5ag2qRzAh;cyWhCb;;a?im4nUwJ+DrgG12djM;E+; z?@KFWtl~o@Su6b-V^MfdaLSDF%l+|;h5EiN>^o!8_MNei@Ana_$S7a!*BA@=eiP*x zBcITaj73%7bg%rEu@D#zp--O1SO|RgDfk?r;w;Jo+fBgMh@HiJ^2=DX(pL}VB%j7u z6y8f7Y)-ylJaQNhtKU-pLdHXEn)8pgkH>7rW4wJp$bBJq^7S3yZ6A+R#v`TEc;u3| zVvvDb!TExNt@FMlE{HL{{ny_Bai+{PmWBE-k zC+3hBpW3Q9Io`@So&7tP<5wa6onBjX|4w%Y|4vW0{X2`mbs=`uLjSt@uh+>LnAj!@ zeJI@~=Uu`};u%k|8xXtor|_SN$F3yqhGAak4LRNwz=qD%yM!{=@+9(%;t}7?|G5M5g+$*HwYna9%0? z&pF`NBR&h9n|T&DPbpmd$`sL~hUc@vf3n%Hi66Plb&0=tl(9N8nKL8g&9(5|mGFqI z@X?=;PtGiE#>d06_~?|)ob$Z_e1BqHd=0qEp}RL@-7R9D{vYJgZ2PBZw0$%W`*ifU zNyK@44WAx4M-e}fkD)nPYXzU{fnx>#N6o<+x1Tk+6PA7%gECnR?CNyOc#=j`M-XDFHYM7gy6%2!1En_R$@ ziI2!=))CDu{wR9rAZ13A?;xC)AU&Fe6HNI+B5evhnd?$*z zXWOC2oAJ)7f#_nGK2WU36H2kkG`BZNnXZxK77~m=UaIZYa4t*{P0m#68FSU4c*B5$3I=A07RO{|){>ZI;;& z%{;C082&vY;8{yKkCyo*WBUgDVJSZD;_L3=eFg7JvUtbWy`J~O@{W(Y__~YF+)C)` z>0a$LV)^_$Hx!#c_=@-+^#T6ppzovOgVj5EO6bQ^@R$+c;S=Bze|9%d26;2bsCWvY z5y%Ih;4gRGXrJ;2ly5VjzY_X+3wht-5#PF7=-)2VYk;i|*j75l_sxJ+d{b7Y8vZ_o zd>4N;Zzk{f#kqM`{*xo*`$JQf!&9EZA8ruthIIPLtR){|tGp8hK6PW(j@%~fqb*wgf>{`{gFA3u@C z9UFyq$%ccc)%}K8o+3f&4 zR%ExOBC8ZxvRkEp!+W*+!(x*^h##mId1WzAA^np1B;zb&J}hGX z|m? zi3j$qVtrl4(;FBxU~$2tMghwx_{L~rCAaE*LN@47E8jdtVgwvxU0-HP!lx8pO7gmh zGv^{dzM99~;CHm`yQ++=kN-hF;y|U30KP@=R^i77fLZwLY~U9E&)&@SW5^TYpNH~0 zc!>$O3%HL$gVd|yCd?YT+%Gc74U`j`i%Wq0IX!_G0SR95H+BKLrg^3&IK)6d7SlMc*g)g(Fj)X!_A5TtRwp+` z$^rw~uk{aJK)U##MDimy@`Kp@AK`neUF3ltwBHjr9KhoQF7yZYArl~%e3cHNU|hQn zVc`VsD?=V|gznoT@o(k!JrbO`CD4q*^Fgn{jAlJK*Y%$r9T!)$0U$00JEu`ip!`z-aIplo;~KGi>< z8?jePH`k&*z#?mr*k-KvO*q~8+Oq@*r-$D6Be7XvA%qQ0yXptkvD0*57**JbfBGZV`{HW81KOS_~haX|8Jvs4JhdDB-i)h8t^} z+6!ZkE&2&>TAWvG(r+ezZ)jNdD2IRC9y=+gOxwMJ_(@AyYwI{$XK%TohP}w++=ZF} zAIPHL2|}Z^FJ-qfPo<2+iaNrak~K`$zIx!2bSXcKdfe0@>s>wH*DD(rLocuaI`X7N z=mk>4^#ZLMkhwQldVxFei4wiQ!okt?0_XwS^aADBo%}%B11Ay}O2(j6?^Tj{5*r-S z1xHV2wd)J2($c+helF`;FK{FJ0rA(B_zXXMaqT1z2HUm>>BwbP0BAy8{l^q zf7Y{xh#jANbH}_^r%_*qhr77Yrz{{ILY}7YE?_N?7y+`D4OaCQC_BDFzDxc+;6&!m zR{E5Hf6Wp2nXJFZ`5nn#bodpjKfCMbljMsZTIpXvpAzX)BmEDtX6)uE6gw;M<$}iy zhj)x(Upf*#Qp$c%`Xv3^>bvgLdir!J`_xhFZ#3YRGSXItmKO4ueUdm3vNyN-HJUns zl{FNfaQc>oTw(RCl)h~<`?l3raB87i>*Bmp<}Ld6GV?3M|6=qeuk&l|YbD;=D(Jco zkL+1LVczH3l0p)P;7z^v;-B=X^sl1Kar=m2b=ZsTOY0^3kVkW{ZI}Tr>SYXp9i70V z4$>UVZ~i~Fg8yPmC^1?#vu}R`T#RHc$o@>`g~S5f3~dPykhpF?;gR;&(=Rzw9tAva zfj7w?9#gK4*l};dhlF+_%WiFHoi~ztMSha;^a7u;r;ZO@GWv=3*i>fOoYX|G^hadv z2KLjyu5{PNdODMJYXn05vbSEP_SW0D zNB2W*b?!%P>*=cSz4bMucida22bZ$`iZ9Lpe$RI;EoW_!z4gIM+t*V2SaWX;y~-N8 z2w8qP@n*cxpvZDoyqOgf*aNb^_NK*qA7NdH!!NEkdp5NuAP22r|9UNZN|F0kq4!zF zQ^T``CxiGRxtf_(A^SjZe{?Z*dgqq%riXK z^Z0lKw#~pK^HJi()Q$G|9|7iDXhXiOpq?4L&nEA)Ji?1+lP^emA@I~guL4&+@gm&n z*H-6IzrYFYZD3C zvVIp*rmOY4O7#FKbLs%g*M_r#gwYGKa+L3p7qY= z`8VO`i{*<0q?^iV$VX(AW=FXG-jKI-aR{6C+WKxVQ) zLKYH;W@E`DARrJj8kWAj8u6y`Rs_gdv2@@4c_zAM=`-&*v=9dCqg5^K9oi=#+aH zi=JZ$7*mi>L~al{V;nGE&;DD0ypU%I(PatAEwmF@UuJ)Oi}k%&^kHC-{Z(|y2ly|% zXBr>jFPEX)1@dI+d=ELcI8HTL;-Yna+T}0z4m?WU3%~gq+wtnSkxgG?JARvQWDq@% zP%1RFjkypx%eT)B$|K~TXA5G0e{+BP{T}u3icg2$-yY!kvHM%Ra})ca z>~Gic{r|eZJsGsWy>{*p`3CqOofJ1(Cr5H zHuSim_BI8NHlTY3?``DSAK2RrL+ov`@5C%U|yQ?|rREzawgBy)9qABMRRqdfVY$ z>fx^4zI_cDVqdV{_A$=~?`vN6LD|=y=lfJX(V=3f3PuhcL#6vexua6o*(RW~ai>;b zPaE)uUWA?UT>PQr9eDs>=*{8}4I>Bq$hrlhKZ~B_^M|hX`9qlt$y+FUngKfY`9mM{ z`$HdS{u+O18+mz(D~^{{FDLzde?gRTb_^WS-ilxd{QisSsF4e;V4tm{L@;m!el@Co$6Q2c4koA^kV27IKkejjQ2 zQG2wm7yd8on`eD~bl=qdqhA<)@A%Dcz8Q^wG@v7zID>{uc+^KKa1v97zMGkq zwAKa;iZA(r9?4=#)n4VE6Zo)qSM& z`^%)YqPs{8Nw1C{pAUH+_?bL_pNQVY_(~T~_rZ1vaNP%7b-=i6*4k!Z)BR>*(-VDj ziyuE#^L^##`Eet7T87uWAJura2= zzEZKfUSdAhS1S6T*o3>;GZlO0)aB@__2`6#mfZE=BcQJ?M<0C0?DLsM;dcz^toy)W zH~mfpp1}StK2zD-#An*A`%1~HyZm3dsp2n%-=*R&O{KhQRedS!qp1ebVKv?MA$DcS zIUDRRHDFuT{iVRJ`%78(;xCN=_Fa6}`Q?yk^w~OK7hU^n=rZaF-~5Z;bT{x{PJQv4 zP6Phu;fE=F?7&|K{LxXsm9rl7B?x_{+rg?1E zz1`Sejp`@;1+B`S+A2D(QP2CEN*y_$5&^tDAwfPY3H}#i|7QBMv&dA;i`Xg2B`(ZB>$r=Uj_dS$oZ_8t2GwgM&uTe$Gm&f6j{TlouQHB!XRRTjSI#hsuSMF-XB|p= z$;Oyo4Z0j{=zqB!f9=Wwe6RSaD;I>VElwpaSkDu)GdGnyF*ZIDCo+%EqX7%kM9P;EqmaTp z;cp(u%_U=%F~%5F>`?8>o-nvw(W?#8j^vlO`Q{>1o!MMSuEng7{O7h1rz`hEAHKxqq66~-owdv12RwRD_e;k5AE|+8qMt zm4|9Kd{4}zz`BP#Bb%X>Bg>B7DcxUsA^4k(52=ut3}1Yv7yB}~ho7rsZJX&wV9W@W zJAXM}?8jR;c;n8CwGrraN^QB|67ZG}-sDVWq>23zyv@_`X6ntH+Sn{U_r;#t4K3KM z#m9IfIs4uMXPfyiG;zXHqiz#_I+fk)!oWeqWIUvBSH#EN_hJspJSiLNE* zOoWbndFsZn-cRhGYyI3iT4GLP!Ku(==?daZSto}ls2RJ-k5I^&5s5=wTH=dCtl_)V zsP%o%8RO;fw8OJh$DgoO%Y3>x|2hVntOmVY37;#Nz6M%Ja_)*zo?7^NfY*Kiouxug z*OE)4VEUuYlH>Y&>Pp$evuqu^!jsMfw|G#|BB=mYGs|av0f!cNaDrH_+QTdHr8t^>sDgY zTv@*RT*~-=m~k{dkl!ykHXYpS?&5A47c$Bph}-$&tIFPfjm;6WmHj(a>yn(|i^v!L zK4*nqi)jCr>s8lM=u2#DaxaeH z%K?7eetaq5HUqyS2w!gSC45c+`+o4X{r>!ZC%EbnT*3DQUvgj0E8wexJ95VQ@zoE$ zy4maFv?T5sdz{>#iO!wK90B^1xtnwCQG9hT|7~v1^5p(*VCn&;0KTpPSHN!vUtK0X zUPDZ8I%D}n@(R;_Oi=rO3u<5RbSCX5T#t+;lq44RJMK1<9oZ-92El&}4f)~E z{weT>`*1IGn!y;7HQ+~9l0ChbF_X=)RJ1mq918&U&(r&Y`q5 zh8Uxni?CAx|2AFU^4$q_GQJo=xfJfN@fu}6H?SWW+0R4R&xf&}4`)ATPk&ABnbP!J z(?@x(Ubtwi#hnSVTH?H1_w4d}Ng zT*?xo>c`C+(8m_=EcF98PN5yaO^g=Plnrib(Mi+UqrS(ka{a#xcPzR(VeNHSC$6nE zXnjv0A8n)EAL57DLi`E$wO<~l;ybH`$25)B)F#p4M3(pL@;(~;?nBNAZYNI1MPHK` zdor@%6XDo57@s^p)A79l{cc&lm0SvS7i=efV+ppix3O!z3BTy1kKe&BUWC7XiN5!s z^k)d~jRx;2;8E^^sjE*$xBP8x-Mw3Lr-H*2aA+cy>FY(*<0Ce+boR=><5Rz%`a7{7 zjHG@yc;CnW2ZQQ=CG!W)rT+Rwtas|=QT|l(tWjQKx2^*Z=U?rcW0?y%FCgP8V@^fR z+7>j&F8-JEzKr*G!RuuW-vXZ$obSf|BzHc4ij8Fsc9{d{EBjf~GXH7B`*ALCwBY;y z8rQYhEl-JSXnZiPr+mR(DB!jh+@BKH)DPe~i*d>LPqj9Ge#E!##lL;S)7-&6&gMwt z3`#e9icKYR$BLmVSe}>r+sS8*o?`4xgI5vL#CaeC{m7j`W^f(me6LMC%$#&24^!2m z`+b?t8V$YBu{Zr92eG$Zn}?~F%HY$VlJ}vGyBNiH`UJW}UtWP@A9HEYqC2_Er!$Fl zvrG6n?e3xv@q1YQM?IX(UBX|DZW;4D?_Etc>o${`wVQg~^zl`L&H8Gj+E_}P*O9-& zRK7bWS<|Lv@|#P?V^!=Deb4>hniDSmg643z1jWm(Cp33*9-Lz zHrHX_Fmn&my~GaaJALO3gQ%-z@QMuWON=tFie=yrqosdcGy!UhsHdWo=zb ze_xG%yZIb%X!ch|b7!k&Zu*Ql{G2}T^^b2i<^JX$-`n&h<9j-2d~V~2PVS2yoq`SX zGxUJ3;PE@Lzii>GOdc{-8ubNF%lUrhyV9JC>63N@PhC4zs6I}M7Ny&d)*)NVoONRd zGb=+=rnbdebe0;m%Fv&kIm~9A8>dDk*Bh%F7=Ku_8U??!uI5=&Vt8iqR%5m3BN>r4 ztBGeOo)wu#OcxnAJbKJ@iNO^8seD>k=G#U!D_JvCUq`&9#H|XRlC>25e-Zyf@{!;{@?GZHY}J)~ z%emz|l~QdXt_}ZBbu4iyCisI%i|LemX|A4{IHPvW=uHsatvuk-tVjZVfDoSaS;g&d_5exEU&{1};TFrBPsEuD-` z{4fpB$rR`$9Xh$%c9c$13XY+ZvxQD36#Q^Hx&J6kI-Oht+(IYxJ(|7)G%ysMc&~y^ zC?6=J(+Ty=)Ei7E-%<9IbVA)Q>IT!v=hO?($$x)*ES+@lKR_o>@LTAli9O*wp_A7% z^gInc&49jUL~jdWU*I0%W$1d)rL05o8N5gP%38NTW6iz6?^f2jw3+-Oc~-E49fUtw zRutB_jORIX;O7bV7;7?~9A+O|b7z0%=rP1pZqe`5E4d=ykvyGyOyD~|ku#fGQfD6L zKl1r+`1bA{T4YjF*%QO;8-dTjnU>ViNyzUp&KCvV6-MTd`}~L*aKT$YPE(GZUull_ zJ}^3Vu%?eJy9n4XWDK(zOCe(_K=;h&th;Z0DPY$a!?oamz4FBZ6~+FU;KVMu;AU_U zD!j^|-cjEf?fU!Hk#-HA&~N(!TJe<@*14 z=QQ9J+4*F)@c@pk?5*d6!y3W$g+5&0B<)})58(Rn0G@M?!Sk4!yTEgj56_%mo_l$| zLu}-N>jG@Yzt(Xbr{lVw|2|yD>A3#!0IntXPK@&k{^ts=!E?4) zpPc6@HzsvLd%auvy)-SRPH>;0I4d>{Jr(&`?)jFzdMx|yQfMXKsP(4v z{C4K6{8z@BJ6Jp2BVwBhSWB`8?uG_Fr7wvoFN=?D(qq~9|4-(#ak%PAWG?gY4Q@rh zI*2X(PS$KbbD2YZIWsBwO}i*>rT!xSS`<6-n>;I8c6)!t==i2oVx=AM04KDW6&cpF zcoTZ~CS;GeuqHz!an#(KvbQp)e^-NAU(Wr}hDf#EfnTzX*xL%^5s^7=gN`1>uGwlB z-Q)t7hE(qOWNmGtpK_tCRG}@AjYd1y^Gv5N>~W3g_v{t+N@SMe&&VGLok?COp*hI` zl8miE&a~@tKX<8}Dj$YUE9Jv{>pvyhzKEO=zk#;&=QD=;p4Ua&cM)U0Ch&X^zql{( z{2hP!?r5Eltl;^n@@FM7ujetZfjKWgZVSx0&_f0Ea6YM)+l98EF&}-zJLmE2I-w8fOZZ0-{{!=DL2hd^`{p+zJ=8At>(R9R z7<27AlPhzY;LM@UI`X`{nkIIHIOi<>yMR5Bc}!wXTZWS>1{*?AIQguh4@*sXzm<0x z@(w?99lA%+$RXa5%c&JvyeMXfccaizS*t~(kAElhRSq6bhpyVlaei8Kb#thXer%)c z`OuXyKqomS|9&${r;{Te`spn|KPpClet_rR2H_bY>}|u?g|g@2V_ilLwnBqF z2Ku=J8cBdw5}~8-S?B9nbDQBG1)TXBn__d^$n&q#HHXA-eVMKtN$4(1pd(plW1tVW zG1oZ*JI5o;kyV1g1#KQ#HitPWVs0*GjxJ)ZE<_HQ?O^V7IbY``QQ^Kh z-x6cbA6B=ZE->eVc*&hI=f>g8`6$|np}knzjidc|=6p1^_+j?L=%D@V3vvhS`N$Br zFn%j@T0Sz-Kc80Tt=MLk2hC?G<18DU*rf1{bdRmOAigBu5ty&5c~^<;y66HmsuCQI zki9KhRTptaxO{`nv;Qjm{G|qmW#e}6U)4}pQ)N^$#GfA#8Q!!SUcBbs!kXd> zY!5BLhP|FLIt^mqaXlO6=-m+R`0DEj$5%Ju&o_qkqN~E^IOqMO!G1CJ*~Jqsp6^)yW-Yc!`pN?ryUB~Z7CumcKPzvt>TAQ!xQhM`qSJnvr99EVBmY-%-ms~c z_BMdGSHc}`@aAUBV`=A3##Lb)(wKrN z*E=%<+Liq?1H0PQz~SJ3s%Gl!Mo0PyeJ%`s2CcuxvrBn~ocv&cpI5=ZUR3Ze{XD(= zhG&VMq?bWwx&)h*$kwgsJ@AJIP3uPJyxTO6+;Q;A2D*EMJ(Kb&_#E~6 zWij@x(b(jr-H%N6FX>y>oZKnm3N3WFu-R_ zv1!5!Z$&137+GC(wOg2@6iv+x{Fd+k|ttSFbu$|*DW)I*uR6Xm8FH)fBDgA!aV+1 zn0L@`FwA1#l=doVPhw>zF<G=d@@68Bb^`0PEdDz&g0? zTm5aH60iQY%l+_=9twWoMZVoAw2W?QUBqvJ|61w@U7tst(;CMhnw~po9D{A*vhHqM zfXx^Ay_`Npo>yAl2ldzk1dq$Xr#!Rju*Wz}5e5CSu7l|rSalk5L(2l=mLcdYHF({@ ze|)$uVBO`z17B-P)!{t)U)G<{t?YM_vrT@d4)ej3hHq71iaiaOUiZUv$~p4GC@cLCZF*oD8W;m$4j7W;zZJ<$iWE|JH+n;!z- z!){mTi_MU+;b(YP=D$`tn zs?5Fc>=K!emCTokJd1MAqI;zBNX(Y-Z@Is?+o}yL*MrA-{j|L&p$KzEXV*VZDoh_laMc*A4}L80F+1-jl(fdWP^{ z7zBU84~3sKb<7-me&>#~$hgiS$MD&mJBlJrohKW^zQ3Q@9K;5VA^Id^5TDjy`jI>g z<|B9R5MPMETcG9~T*-PBdXqUlvctDOYzxp3_K1_gzBLGTbqe|k(9Ngx@q=jpqSK@s z%AJaC)&#+QI&}XEFbCk@_(S1052Bk9Ct16Zr(V0)o#7aM6*Pv1lZ;_Vx*>0_`N*;p z(#=h@CwyywZX}QIN%+q6AlOfrZmv8D%qQiU?6=3PSK*lg--&qUDPRu?g8g)8CnE^& z<4-b%A@PphZ$5I@3Gp6181JXx_wT=bl-Hat-uvJEf$Nn$^ccKzKIVjb6=z^iwO2h8 z1pDdW{SDv_tl7W)&@o^qI<38ma;M@sx1S;W%|Y-#dJ_1Dq?`S}gl_Kk(anL{)Qzx zcx#OgZ@BV!N2-p;zWe2lWhQ0c22F`9^D*BN2Pk^1>pA>RJUfKVJ0S*JGxqg87ul9} zr)#<&LSk5M(A2~3Cuci)unpvOkdp>`YSo_7rL~c&`g!b&wZyq?A)dL`tg2J6bIEgw zZx*@hd29w@XW1SK$5thF9eF1A9htP%&3Tckubfx`x$CQlZ#h>hc~on$Bc-q()?yDz z!B#m*-Ls%pi>OM{9&f%zYi_n`ziEDta?;kDyg#meVuy!ppEC@9JxM*2_Q)|XT?tHo zkn*DIa_?wqJ~15JTWi*A9d7V$o@Q%zGY95sTfZAzn=Y_fUEq7Y)QQ9X4sP-i$rmf% zXK^lp?`Qn)<&d-?A}+IybNcT{9@X6NUcY)5|VXjz@X+J!|j4e9;@?oU(SlPG_ge z$u2%iMPv7}C*uc{_}nF|hp(8&PJC+>Jntc1x>!qSS_z&u8zOsq==V>OQyg3PHKUuy zgmDjKDd(ybcEK>Csq+#uIgU6_v^rZks)*UENK~W3wVYY|i4E+>X2_o2A$Iv%`m;f| z6X-8IT=Hn~ANVebR9(_{Riv^v(Z*`cW?h+>I6XpdBmB5Fgofg1BVwe)CYyS<;19hz zT6JxsjreF~f8;vmmo}1FkBz*`u$nT3J_9gp1vlO3G9Kok9lLIKjKuFJI!&grO-Y)n z;Y>}3d6=pZdJw*G2pW@^pk(eqkU7XFUs!3VswSURNp}NxY7j3i=V{8ZBV_?sIrkQn zk5V1AJ$X{E~6fmmB|yv{5^OkH61~pgyI|He_=drv}e1pp7e`uRu)V z{;Mv+et7&@G@&VJUt;Ryx8Rp^Dt38h3Vz1C$uqGP34A@We*ExM#)E?qs^d6#4x!hw zZqp=>hKje_s3*4F9&|HVD?`C-(bq?e^CQ_KUQ@&;AB|@$$~7&l$W^0=_r1LI*Q(mY z+?Wk&pJm-7J5GciTI|Dz4 zDf^5&9s47EqL{Uo#e8LDC7qh-B z)TzANv~bDYQ46K)e)0i!d72E;#6kDp-@ z?FC>lgVQsi$za?+Pk$5q{lP!`=8y-Xg84Tw|A93kchoN?ZajFdCqWxB*FEri$>$sA zZ-ZFwjTOun`^`Xo@pTEm2*)R|9hnSx9V02P)2ggX@;#jhA3}Hk7TmFhR)ddVy2zp* z)Bfr1Ih+0aiB!GN$d6 z)A?$Ys!2}b)2iCd;HRDELTl?J{=?66%&fIDncv0!Hg>Vs2HN1xt21ij=d_{MWz1&U z35O;u&_o7nJeYnKGnPjv>oUU2m{(%IeB**ACFk;aM=ec$&^I<&Q}R3TTxi%GV{5N? zNVU5|Z5Wkvs}VO&dHrBMq_ zla$pp-PW%6OB*JCp9Pc;jA13~=|p3AlQ9IBwICCpsO&s=;4Ip$V2rG(WAHtm=i~?J z3|_OsV}$lHcsH2#WNnHJB4;S1J+YxzvgY?1EzU+_FGMcMWPXF^elKlFuKRNKGm$Ud z#z{`%H=2Y-wzC#=n8pE@pT14>A#-X{$|L!mhk|b}^#VMliZc~+&q|nHnIFl%8s?0k zf5CkqwjP;4_j^kJk@WAP{}J@h*_z(TIXW%sd`Hg6@8;ZNv1ab1ou*ZcOY#B-#^hu# zdV#&DjXi7<{mWVPPRdIx&WYl1Lf}Qgy6FBr%Kj+&=bh-EA2CJ`aU()k2az)m5QikX zN&3a;o0Jo|NA%53e6Mo0e2d&?K}=^dI*jN-fw;^$=$?7-s$J-q-NaCKbM8&f0QVwC zNtrVI!?Hd^zfJxlI<3?tepa3@<@r!?Dg!YpqRVWFw>#0*w>y!_fzTIeLukp=oJzG!3>eNB=1#tugpC-)!`g5f;bK6kUOXCS&S!wv8@K~PGrCU+_fSD z0{6}APf@@fX^3<-0k@o~64*r!fG%_yaFQQ(p*7@{&U08JL*X@I5WH?F%N!&h4sp0y z$E+)d?9Ik;`0g`|j>r_w)`I`Ra`&9Oq82WRwbe|WZF7u3X69a+cfRhLtmiawasI8x z80~zJxT0jkh^h#5zbN92l3!rIU8qXz$mO5b+`M#JSV&WfA^C&ig=%9jG_ntx4naS@ zoOzu(8+|ZAje6=xUiM3i)W*xfVHEQr&qL5PS0^M+fBkS?c2^nikZq&Dqr5XA+lv0z zb~E?Mq5nzVCHJ0&1<9Njc^>>*fK`5{5Ys%z|9k7D&`*(1c8q|A%HqqpR- zzt_`NDf-0rAM@-C^oi-TU0HDYFqh{a3-kIP4-NBS3zN?w3t7SaXZf z16Hx$5W^8G`h~ef&hPp3i*Ubw5y>2&|I~z|Uo3{FI-#p<%CE}PBF-ah8A_#Cy7vzqUkD{jnjU!rEn+AihXsmS(Kv@!(ThVC+PwtC; zIG&u)z~6xLJI;y*^tjCEqk3E__wtAy7mV9Cs3UsZ9`?3&>dwJ#X+(D(!F=LBjCD>r zeVn&}L!X}JTWeny4~=ut%~sK_tlt5;4Wh4FzrL6W{j32dA6}!XcR@qJcrE9@j@RgE zKD^dcEe+sR=G0Pf;yEp*?Yn5(narHh_Z(>Z62>d*$#Tw#*Ha}h-za!WQls+3#ylB& zD14?H-XQasFFZopfQN*$7Ris$Wuv@2Gjoo4V7!CYsnFQf0#Alde-t_s{Di~bBGDy} z;c3wJ?{q!r&E~JP5zhN^bFzh>$sSXxp=+`~KF(O)IQ#VHZX0*U2G8AO+8l~c?xbDD z)VGIN&Jm13_S#I=!1eqOCq6lXbsNdLHM4HTev!!-df6+m!^H;pX#NBrt*@MNX0(o6 zJ@BihD!~7l@M2;P_4RJXPoVR)A^27gwDG-^llHmC`b^pnURzGu*+V5Pp+=quXk;C_zK(zH zDLxbYzX=S%W6lDHneS97=vf zKJ?50mQ3s#!8i#I;{QBcO^V4i*kWE+lsG2 z=4^Y^na$a4jH3}aM6ZZI=a+SXZE%6WFYPcit}Lx1|3@Y_I!!ecRG7khTWU8aS?8=^)A z?b*nE~1PMSOkG_DXCE z(x%Yiak>cYm5WS{eX0rjR2B9q>;=v+c;^WCq-mnqG zBO|*TMn$QI-;KIw-8TMA>_y^CXhp3HTsF>b8sjB2&Lrl~7 zBz(pbRP|f%g3b5)+T4-)esirMsz;BTzXsbE{Ax4#^6Ib$Y%q)rC;q2e z z?Pp@QXAgLmvqBrlr=yRF=NIG8`j)ZYj*a6x#yR~m<#>OL8WpeQWWSey9nqw^HeffD z=P!rYtnZIWoPN7GXO`r(_>4ZkV7xKdRBmP7T9~&+#xQ-qH)`gcw=TVfc&RdMDHDtNz`e>t%xPRwmZ$vH&?qd^uw9&^*Y%K3dOicv+NBH`G+GMkCj=)}n3~sUx%M_oj zH-zn*vEGBPjEsMx0WnUg@-&;5M{=wTf^@I1<*NDny%6EZ~G zdBZY)#~6I{G)!`X>A!{UWej2`zZIHnfhHU1`xahT1?uDn)xVN@34Zvbemwjw zHuyj9qk{h5&i{y@|MBoV?BDwMr2c;5nMC#q^tB)R`}=3BQMVZ4ng;N@+glQY448qP z)!5diFH!gxLX;<*`XbZNuJ|}=l2cUsFrEwlB$tnFYc1j~2;N)2n_mcDDS*G^ zbAIUcHnH(>4{Ya5zui#kJgq{T#^2?$<9-Q@Qjaq&)@Zm+|qxUH^ZXTIz?@DlkfJS)ucl;E}yp##8fQP+LO(@?83t zcYJq9y|*bNeBI;!-bTN2hg|SknvAKo8uNsE7T`0j+6(W&?|*r+#ZcXd9%hCoT6SgI zSHb(TYRd7!hLc+%`GX8(_GEOVki?>c#OIgHLH-MVrm-(rTJeD+^P7Mf+;}dcjOb+D zdR!W@X%a`K|BoeSjW0)%dxFXq**Mukjy!bM4082YfH|9I7C($CjsE<<%ixa{{@)t< zsSX2m*Wq`Tx)t<2F#wyypGm9&y7WBwvcTUBuay6(=vx^nHtX0yFtX1c2jlis$??NJ zOj)t{NZgW)X)|TF3{$h@OhTgp{G`%8^Rk6lG3l$0Z;`D7FxGRgq`gm~!EoS@Iu4GNQGWQ*&7{3@;P}xXIL;lS-u4lG z`0+(cJri(D4T58<_^Snvz;W{+IL;oTy*UxImv8Hnd$djHo7-qFDX6`Cc(}AThxUFl zsJ*Zu+S^W!E1Boz=wSk1Bkh?6jq9*F9v|q&KKzFI>J@>HgZK}F@rbT?93JNk_tld- zA#{9DZ||Ub-9yw{8RDx~AUq8?Gl27d2Gx6ah`G_?SA5#C|8$Qtbp|>P(hUt@N);e;rn>A-jby+5L${|e;(j3VC|qpk3=j12fAIS#Tklb%uH2C-wOQBF3_$-pN3XI35O(HIl?IUoLhr*d?!K#nKwZ!7zE9P&~&vQVpGXKwdMUksrsl3XZr zY#s1f`>u4(T)n33=^p&ToC6kruBe&}P;QgJbGAqa*V!?#!TF*N3yoPsn+|66FxRj5&Nr%1PfvnfhLe-gInk zg@C{M@8IbsauLYgQuDz1?nN>7J^xu;^D}Z2?c1t#Mbp+((9nhV*!q*TVZEO{+A`+p zl$LAmNx?_>s%D?YoIQ=6xj6(q0sK9E&^tryUwzn9-#*~o_;qA-)7Sr1;K2{m#hJv> z(nmYrCXokNBcfx-ewrd{hP8toUSB)A_|{lEDd1A*T=ue7ZA_qNqjgObQ>t6 zu^$ioX1xsjrpzMM|4QJ$g2tsy3(t{PHj*=6kAa*DOnJbtopv)#ob|?@)`z`g;zhQ0 zuT4$69+?O zdkD(v`z*KMC(cy%j0raDlr)<)eT1#%Cfmr&X=%3h zY5%iy$CM{VI-^tnz9s|spfMfRfDP|6YIfcY6F*ISaW%fWl3Mt*$eyCF)q$sd@Zrz-$ou?0Z^?YdwlQeDGEVMxvj2r= zwb%)4z*rlPO#u7EK;1Fu{$JB(5;Tb%a1;FFvLu^zc7n}%p~+VBlr1FlLULn1{m-R4 zo@xzoMy0*JMjtn2W!&4LPxQ8~`MK1|p-zIW<|6n*0lZ)8{>5MK)7013{0{h53oP_i z0nJ=bU-RkfGWwcLU+}Va!PisNd9XFgxer|3M4eiHo$bJXy6~mTIuiK!K3(`ynHPa= zC4A;|V0!@ABfcWuttY;xSgT&d*c&xu8^&eQEdASl+yBZV{Ku~n zdp@#G8t?A_Un%@=w3oK8+*;Z`XM?%D%^2S<<%sd(EP$zLe}lnuXoW#PukB4A;qb2D z3{>q%hYg<~yysm#cRqc*!TFFpzg$x)dow!NT>QONsoR_H&_+AoP{cnY)0h~KoYxS4 z?>@%y;Vs1Ra_%D!m~2Z`iJaG{KCoI^88k{F%!EmaCW&FyNj`x;sd>ocRf2)$KFra)2`J#-ai^Tyw~&H$#)gspYVMP z-@SY<18Oi>wiJWG}1vz&U@GgQ_skQVm4;y3WrD`MbW?f;l zE$`hzH~x63jGwBKW#^kR@5)o7T2}b@>0Jxe456VdtTD$WYNMRF>SUifpS}ZhS4rP< z>HGX8YUBAb|MVSr7f#=m^gWlp&tJ@Y`i`gX!23YoML*`wD`K@und8)Q&Th@k_w>)t z_Z(c9@96=zw)$6d^Xp&BEvSDzcUO4S`Z@I%_Pb*=Pd@kJ?J|yBughb!vCBArea~wq z$}cC5^0M_d?&vJ2zbDrnlTW_DVg32_^|`LGnumq#Ihd|_T!kUX*^*m(v~$-dwvOF9 zY#lb{Fu)&rdB69ya638YJqNCzEpcUiwx3~-v2zZtH>9ItD?ZHW#4N92Jp$h%;Oh>J zTwhlIm)!1KxVNPKqW+#<#k;M%dzE()1F&|7Ub4Op`09bL`<5*i)Yae1`*}PA#$}r4 zz@Bict9N!sqg}NZ;Tve&NC^wnle)cY#h5YaAe z-~Ctbqix0?>pZ7u!@Tw)_>b_eRQ5?{TE3%oc>enC6$$GdY1nesYM!Qo+15GCaUSxq zdt%F&R_>YT-duM zwN=gc@%w&$zop&R+@gh77Du-9>(MGUTpC6%;kGN>tG?DtN4#zyMH?0 z&uO0XpBrX(KS|%u@_8WN<6e{Rc|&s0OpJDTvPL;%{^RPE-MdzKyswb!0$SU}v#Sn! zA9WWNTU{3yTZIpZ+`fOU=v69v|G&N1wpDDK@yhmc{G&x1=CqgNAH4(LsGRpI0{7(x zY_7zn^h~+1COqBNz86}q`tO@}RBjD(W~7=p(-r1iOxgE^hi9pc2O1VS(xJOU*OR9T zzH|V-)D4f?b(QA%9$uD;A0eJ~co2(D<+Fi7lQ|e*r#=t^?p;Gy0=#B5@<4gI!7yrs-&)AORjrjzUKh&eGh!yv+6IXYpB!Lm3yM*$&i@O?OOFE$YtDb zQ#A>`mHd`gt&lg8yR_<7LO&v04%=< zv@;nSBxhfBSfH=p3oP*Fl<%>30Mk~!19kpW>P)=a!Sg=XO_Fnqv%NPd&j;}>%}-G; zQ1045+doAY{MO$#G%Ip$pv(fw915PZ>GcT`%9u_=di88iF@?gr0=9+OR2X!=@no z*kQ0CV-T-QIR|!y?q#fP=3e5>HE@{#efG01zdzsLspVN6b9RV5rhEF<3v9;axqIdo zcy@m=&)UuU;%*jV-(#H{LWoI-$?tGKcex{3Q`LuJRyH4EUH23kJTDtncF(oSBR<*w zJ7{ML?LcS8w1XTc?W_y5^P#Vue%c9!MQBaV(+bbnuT=IUX|u_9$9=!VwWZwyQSuV~+s3=E z{dK-QTBl#0AM)4n=6lTKtI@b;S?cuh?jZcC|96@mOYWWzKivS&dWH2BI)8miXvlh( zF`s<@Vg0Sf_i|GjR^_^;utq*)4Su3|+KdbPGmIm6_ik>A@xI*sJ2j6xoiTn*8{hKD zqU-@FcX@wC6Zf?l-^ooWWX(^!h#btwP~is8;gNh_Ot~1o!HsJ)-*fpMi=6OevBRB= zoSDwEi3ZPs!f31OSzCwe;%KWoeV)Vhtf|Iz2mcrHJnJ$`&sBV1E5GMkdP@1ep3g13 zzk%=bi)%8>*c--P*pZ>B4p$ZH9{ZM!Jq5jYWAZfCFYCNB1cItmXPEF*}z6#dT0pK784!JJgX(lEhFCUu( z^q)6OO|!w{%)nw2T8hz-L)mw;*moED_FJ*9Sg^0mLY~YeW?%*~<$1`I5+mwjTrT!A z+0Vr`X2_{qu;g|1u=q(upZbzBh!TfsU|q?%aQv&Bxv5)_%ljn0p>O<=KEK;RWX_r) zpPQW0UMcz?`*cYqdigf=@=o;fH_^-A^4n_O3bNHK!B#U88ZFf#dkgWiuWyL&-;ABc z4SyGT@J(zqs}j}arEzLn8-CVY#wh2%6X?f;oiz`h89`s*r&{ioMmDI<=NlcQ&qB^c zi4%X7`4HGG^fQ-wGCmobH{DRUNR^=i>FxQ5wc!Su*Qd-V2yk8PQ1qZd511) z56+d7So){J1?_)8?+`i?nh`t`3o$x)p0_j3a&!<= zeUj6ZymUhX^e=qc1y6DzZ?p+-LVniBPa{0$Bl_=ufSk<9YSeztBn4#LySWcaV%O^M zk>p`}T7=y~rw`(KPIjg_gEP&ddpnW&MUJzuRwOo5pF4DlTx?lh;w((eeI~FLr+zr5 zGfn6yys`{G&1Tk9Dz=hfS{B_v=65CS!Y@1ro-DMw*Ie4}KrVHEZR?nWT)KZ5_Uwkg z=e8Oj&2_IJ?>y)09mWJi`mo+sr?j3` zQ@Nu=Z0DmXKaOvavn9vqc-~*dvlPB9*gGYj>0^8{SSta&ZBez~6wpKg&?# zP1ZVwNg2j2zI36VqdX$sc{9(xgBHk-J6iH52rnbHSf770Jgo@($2-{EUNpw{Z$H~+ zePfKx+8O3M-`;7kbyP9FO3t@iQr}o3HV+xo9O{eCz8xF%%hcNkpA3_Imgl~;8|93q zzuOV z^e6U4eC-=uS79TECZ%3RsLkqXP#xtaH6w$3;$m+Y-0yzc{gU~4oO$ZyTWGb1IxDf= z$hvz2eOJZ-|I+nVP^>0PEe znnKi!Qe>bW&b0U3p*rB3eXa1@%;5RIMRhy}F2b3s-dW1i%bto|>GCS_&Gb%J__xA) z4b0ndu*|Y~1eOiJ@+ZbAzh4{#v*_(I5Ax2zyMOx2J;U$A{@+jXd+(Hs9Jj;QU(U*Q zghP9qr_6F}X8rXdgVr(++t>p$iU09VQ64ikY7=GVg415|xB%C?y>bUGG_kts&K%;Z zXIw+>lGWrJZh=N5E=XboSMtA?`$Yn=ln(m3libJ-<5m4V$UU+TZiSyq-dSYNeW`V$ z?d`qjstvyPX5gKZYF^vRcUHA>oM#3%tk)i3asuniS+_gnzD0qhl6(}h|B76?uEEf? z`YVyIFLy}Z8CheJpTkW(u~CVC)Xh0f*}F5?!vp*GOvWd7*gi}<-+@2Hm?h8b`CEMX zZsq>L89{xxkyj5F&bGG9)v7gQ9oGXQXCa4h|A)w_x@?t3?vgCkA^3CEhUjvQD{D5f zCSgru7?T^B4m$TN1Al*HOvQ^B-$X?_$Wz+bUa^A*#)SOW;ZUvQ%`|830sc?H|J*)g z&RZBKW0d<+OOe^Tq0csOEcU;3(B1%kQ%2j%{P=aH`*Ie^{Khfva`12~a?G8`F+xAG z|GL4Q;NvpJEM;7!v#suHu>rDwXXKkQWiRJwf0WyO4bT#sG$G41OpC-O< zwj+?A*h!Glh3-NFbf-D~fj%T>yOHnF6DTt=9~v|_^=1_~k|vWo1sd#L zQL=vb16Qm!V#_jM)6%d_DePE1(4xNXm_y4V&F&sa{ZQ_Pq%Ya8%h0_7u<`sTjjoyP zID(9Jc%l#Y?sS8rMb0&I)=6wO3VWI+_OunT>lJo4P3&$f;?@h!-SjE*vz53fv0>@_ zp?kD_SC9A6zq7}jEd^pl?XZtUXIdEX1o^zyr^;`*Fkc;5BIGV1>|V^b}>EET>dK2|qnEIjKmraOyhlYP+c zNY8eplQSlOr`?o!WFNQ@9z}lE!1@*Z*pRD^#ZPn)enuUGpIz|J0sPn!?7sm&`WzFZ z;C`;aVS;9h#+%l7PnEvkbw_MO_d=Q(dte0qX;E#F#aIV*z=y$-zj4o zs;=F+kJ}0(AZ+eJs!;@z-_%Hga{1$!{C?k85=vRSyI(-hoYd*<6 z%4-_oHAUL7yk-aQCC)R@m)ML0Z3^!x=Z=H`pON||?4N`6sucK$q*gyYDd3j#Q^KYmlvZUVEAb0|LW9zHK>~Fe&*Wd0l{&uHRe>e3TB|Ze( z<9+Nm$+xi<8Aoyr-(su08(jyvOnfOT+lU!$BW6_jVla{`$cZOqmS9IizQ|b0zJcty zN@6`y=g5BFTL_LNpN8T)j5e|=Ds!qr)eKp4!bj`jAJZ6{$$x%E?0bRsEh*B5QMaK; zo5Ek_CfOb;zB_!awEYD8nApEe#E%8eOUM}-X~QgM82oVRbyF_X;du=>1)e&-1MrA$ z5)6;nVSIIp^g8L_Md}d8Hc)3SapA#r0&ooOTksZ)Gij%o7@kt})q24<_6#?+;0$tb zWs-v{%b$bGqH%X6I*J7yB`kIO8a@6Pxq2n`;Y@j#;)88BFrBFHOh3$KV3^C9S%F#j za8aR}l8Id)Hmsxw zyvu$|e6u4tMB`)NV{{}FJC)oW(yzzRg{y(M9p>t2z8QZEkD0lYd@~yGScEQqgiRh< zsGjzVzx(AwrNl9|e6ZIhWG6hjsikbzh=x z9(CKs(H1zpg1C=ql($dlk{qGc^~4jy^WT*-0ik>(jv|DQ$QGI4`$X@EfqOTK&-lBY zYo8_;%@t}`@2i1zPR#tBj7jKkB|Paf=x-BeV_uZ}9N25FBc}MJzho3m0#m&^cnX_Ba@tO%cn)A*&S~q#&JVP5hX``Gu5S$C{gYmv@P#fn5>VM&@|19;r zQlEJcybJz=abF))e^LN`?6cB0J~VwE8kq;dyWl?<_lx{>&-T|de(I~YiF%uvziqr1 zoD1%sX3id-@R#Oq!1Xpi?uUW<$M|2wew!@zVfx7+UW$3`n*;x_U?U7mJ#`LDu@{@* zJ%RH_vXANQuphgjgE00$3%0Hd&Q8uDE;)>{X7&UrS0Q^0zmw%05$A@IIYT6E$vgQi z&!vpKljnhY7IH}TViR0~J#jC3+$!vel8>o~xTiPBF>7J3?d4NSTvQKn2od;xno>KP zu`g9t5NnX3;j146^Cj%#ORz6a!d@nAWn#}OO03yX>2I%-y=*VIcKdT)v9ArB#WMS0 zzJ&PDCB#Wg!Um|ffeo-I@fRCbDIZMFF_uqgBa)c3CE#Qd_CCEFHsGSfs)2G%*fY8) z_ZBh1OX4UO=Pws6<$g9$u8MO3f1@0-=|*C;x-t`)FL>kRxGmNfh~F{YH^R9@>!0Ps zt|d5`i;nw8%ImUC5d0B-_{mkZaq(HSbC$oIM1MPu5zh6*cLmzXl3e<JG=`|%=dOMzUH(C7eN1Zde$=L65f z18|YQFN>J`Z9(u!o<(^kG!?w2L@u5KU0{o^-pjfw1vb$|N|7hl!Jjx!;ow=HncwE1 zd@+ODK}M`@Wqpf`y^H!8a^?{EuaMuTv)(PN_rYyS-^I|XnRSTVujh3ds*Ly%%)o9A zI&(d^oB>=2zGS??yn!5#1H6G;@#IrwytG$Ad%f_0a@w57n#zI>iPxCcz`P~nyOkJu z6YYtP7%ZbmT)fDuMc68#4UgD;%_+*7d1UF1uv}%e94X&neh0pT&LlRRmB1x2daJQP zNu0cY4S7UAYk_C31n={v(f>4P6&Nxn+pLoa%KvCg-)SQ{k)~nW4X}aQ=Azn^ygfp zjKp$wu&y%rE$c`4gq*?LDZCvx=fiWBz;kY-zxk}GKJqGN;a8aooz_BMZQwqYc2iS2 z^?3I@Wvo;@^D@rVqc|K`7typYvbrC)$LtI6Opo}w!fEej;2F-CUW7+qh7Qq)OSpJm<7^L^lNJ-nTGqiH5^Kaw_F^^Ie&d4}Iqlf1AmF?p0`?i!QP8a?)F z%`{iyR$Rsh#*D-VV?wuh0)7yHOJI?>^Y!Qtk*rlihUOT-8JF-@Wi^Ath!R!v0`IxE zoE-8)-_1UevrjktRBb%xy?jUJR9i=Betw^e*gE&fu(=lVS)&-|LsjFqH!s#qof$K1 zju|GK=S6Kq=d^chR>?_G09=LOVjQ2Bh^gr(c5FW~x#Xeu5+BcAR^lc0eKGQ#mst2R zeDjISi#*G}T9uTrC$v!RO>&vENR0DTVjzewuu`T4eJJ%R)=wyOP2Q|8I6wX}XC1PW z4373uwxxH2`zxuxllt?R$DL9i8FZ)Ux+V0-dGfN$C5`}ih$Fz^OlV-kap`j?dmx3f_$sr1VsPMh?%Q(-vL!ytSMXW3 z%ELENN@i~HfzUuHtQe8*lK=Un3%aK%+|4q za+{t=bS9*~zvd!Y8^EvQ3;1OmI-VGtM*okIKT!M%#4Ws=2(JHNh;OC{vN2xIK{=JUK`&hPQE?IWXU^Hj+- z;IVWhu@s!ql(r-mFE9_K;qd5U-#pBjO`Zbg;aTQkBDj2odEg9YY+%lr|31NSH|69E z|I6g6^yAkmHXEm^T0YuFgSKfPK7*GBJ=b`Eb|fXn~zQB+Lq?gZVU9@%K6|x zTP@IiJ>yGGW{u2XKE_dJg0IfTH0T|gZ(m?rD)hc-l&#%I!=s&#rM*9~izcj_wp1mYg{0^!?~B4Qt~Jqk9WyVw2HEH(kxzNoMWL!?wX3 zRNFZFkj$D{MA^;ImW6VU8EiEY7xoP6_G5Bz^sD&J;hbZz0{21INh#~IbvkPjT9#NV z@nJt>NNV~Y@?_NdX?Z7Y7fn(ne}$I+n(|5W$12hJ+fg~$%b?@O@za(Xlk_$IQ|7gu zeiDdD@X*hB^pjJ5v>zLIW4#RY^JnQN{MddnGLH81kxFo$Jw!j#=tsu$5#y15CP+U) z<59pW{YYHMA$>e!_3^X>j|V>E8_z$RrJvujHXmm^;&(1Jj@8FAntmQjAs@^jej>h& zkI026eEnJ0z0Qw@+jLtY^U>#K9)f9}J)?aeb>y9ycm1q~XIT$^KGyybxd#RR65sG^ zmC(5#KiVhol2Sjswcvjn`2&8g!s~EPQdDfXqEc} z!TG&E&odK-h_>JF_x1Z@Uh`b`v-jF-uf6tKYp=DovDfB};Hz#b^Aos}*EiM&|02o? zRtr1bJBhg~I){h$=h}06WAgmmuz(yw`i42v#u}Ex+$*7O^zgmjJY(Van7cn@y;{Y3 zb+9zMu!Z%ig*nmkOm^sv+&?`bzYNjbYvgEo$2fJ{@8BZ|x?c>w4#(-f0p2qsiSA#4 zC)sq*nkBk#r>yX>mH&I7`}sCNzPexlwj#OBwH zjFsQxe3L)(ud}*!es_#=Rg{alvIo|%_8o*yf7cb;@KfQDIUa!bFn9g8fSXOD=@Yz? z`>rpGj=*y+a6UXyil)no+-vypu&)xaNMCvZ)+d5>>V!q9T zU+UXh_{Tf!F&t!%;V^p)8lw-`NB964`h#|T`^P8j*m>XzvsVpn{VCKXxN4pIu6JCM z?fyyD6#U)nfrG1|a`T=0uE{IJ!zK^pj}gu%Hz(GYv!v4r=g3o9_e;pVAe=W*?jSf9 z?BdCa#}scb;kjUM!apwo`yjB-DmxMOKLb0mF*uz(kc;d=T|+RjA*uT=jmZbB?XAEr z|I9b}N$2^6E3LT*xmNp!_Vu@2mfkN%cK(3zI+y3VR~dct0rueOJbTxe)I9BkvT30+ zc&=|GL+5hc0slIOeP7{K`p5^!iUnp)NV-Y_hDk}+AK+h*m3$A(@~KljW2onxr1sKz zUI09vFB(=ha*Nt1KaKq&WLw#OCZg+2Mt)YA0=^r>t~~kI*pKuLx}bCm$!HT}&UxD^ zlf(04_kV#so=e!vxS0GpoDF|lDL$Us*D&pvVfF7Va41~LF0MVarP4FdPX&A8ejL{y zFbDp0LYXXkz9h;h=YVw93gDg&ETXsL@7VVwADOr9ILzA5*o7V5^H6{od{5};3S>Uw z$rER3D^Cq^7K42zepP{-V2YcZVcnC1Blv$QFeT{5KKb{210Q9NeWv}2?-(u0?~sUx zu3;Z^zTp{AoObd1%t!VeK1;tj)?fx_8=mG#dyU||;4bp5!1A1WpKWtE2(SnVm2)<%Pm~;5bl(DU(N;f}}ES@L`YH{TI(Te#c68if^B+zT`6L)Ty)X z)Q=-3`AV=!*gP4Z$T0O`d)aF;R_O*dJ7DbQ+3D2F{qljoe^>s(OJ&!{ahq%1zzBZV zc`g|jOvk2KU>UZ*O`&NFc4&^c&F0Hejh`(&Z}ne_jljMNv$=H%xW{MED*h5VE%Hq4 zEiOwo#&lw~@cesd$z>ZkV9P8k}3O~kj(`?56KUh51*xLw>w5J?l zj-+OxmPCB12Ml8+@i2tl+1y z=*I>|d)w3b6=M@C2hLpcIQ$uWvlM!kuZ0a4WxVuR{$Qo-ml?F5=&P5qKFZc{rkf9% z_KBvw^P80uU@>%G6Q_G42^R%ka6#M3jZnik;{Tv_-M=teOz6x74eg%cJNQ^!M9A;fUo}1R&w)fQ+*!BEuC_OdFfA8@V=7w$Lna=-G{ggYQe)z3jKXWaecXs_f9=m>UJlmT}pUHFBpuUo8$nh1Z zThPo{)S6ALBhN0gmI8k*{et!frsiFKWD8?n&OU_)-20N&5cZ#KSwUkHBli60ogU-_ zFY-bE7i$zheA` z@Kcq4N|-gcKdt2Fx;BA%`7ZgRIOh@m3S4;@3+aD;;EKF!SW)~ld#6A%_;2@*C|#~S zW7)9bAKUv&d>QQ|B zn#kQQ8+f9gUGS2V*0a~4os$!P{N?0&U5TxpxnaK-O^>2n;{9{>`}9Kc>5>mNJw(i$HGpsAnjL|`XU;6~ zhZ(Q88+86pR_Ggt@F`nL-lw!nbvEHLgE5PgnZc)S1s=|3+%<#o)fnQFjn2C8o|VVP zu#a`AXN0k#95}+rWPPl$=drGM+$BFx(13W79p6qLrvl?^=f_s{vYyyuRz2?%JY(w! z-d*U(@>{0{{QX-@e}q_rA=rWKZQvK|WrF=$V1M`$8+O50^}S*6;X{}Ue39Ko@K*YC zaA%pdYns!iO)k%X_{KMf^{F1Z4rc@g1#bv=CBHlU%L);!*V5q{-oGryG28CnPbeQI z{;x`BbB=M&<~(-)70jp0nOB!Fzbcq#^O48Pky+igJiav)z@InGdI?^;l^D4G-Ny1Q zy2b|FPs~L(*SB{DtYljLPW)LiUEahq(ShjUcJ^~ta#o6bRYXIHXOHsi9(_A(ZY+sE zORn#0JhQ*^2D((&qr}WtB|TGJD|xn9Gv4#}#3%ilikDt$6 zP*v{Fxx!r5hyLENz{%|FfD^zOruQjZLoR@k8 z+SZw=F=AXaclx-$1OISh;T501tcUeTdC;y;H`afIymwQB5$yO;VEI<^k)t#FT8+J{W3d9~+vroX8^Bh;Utw$Xa0!FS!@ipGUrzlXWIkc6-`Oc z-1BUSKk^oG_L?u)^+hfW42ri}?2}0kSHHc!K-X5{3;Jin^WpXIvu%aYbNYX{EY+iW z)xMrpxC^5%r$=8raA%rz5LoVm4~H9Gnp-u291yHaOU~!nt%h~aEv8jF)n(niz-`@q zeTsE9Z56zA@8eU|b{vPp`>8fKtEzDRY2@T80cUFrtK$}juIk{KInaUl&&Kg3gW~_5 z-GRZ0(9w(QI8{Zqiw9$P#8&s=C)HS`g`%C0r^bEZMQkbNaxT3eL2qj6VS5P4~^ANgzmJt2?% z!jaJSE0TG@f!7P2Ab-`tmvujib^iOvYJW7-n?HPD)Kllyemz=jj2);r?+2}qu+ONn zlGR31?(#E6KK1B%=vv08fvLHU?1L}-j>hwm9iY=Na^U5_q_XoV>!Yo7_@HF@UCJQ} zTvjvp0@kkhE=yYjrA*&bw4{a4ID?{oqcZPVcM@HRw12rUU;aC)0FHo z30$2eT#-XjaPkl9yus(66)Y35cK{1~UUfDei)|>R-v#l0 z=P_1lL+65E8wrNN!L~1et1|dEz9m`us^ZM?&u~iyNou8q(R$%ZweGNYx=rkTO zIy38x$?&3Sk(89z5AKn$_jsb?yWFU%l1nH{Ut27P)Tt&A2F7=wjpU*fC8r{Dl_ zpey))pMS=us{s9GDtcAgLqS*EXG4z`lK5~~s4t?A_2 zJd3uQuDN*mBiCKLT=HB2`d?;tVDK*HS@vZ5!uy%LFXH_~$`8k5)TVo4Vw3PV!^Y!e z@HlA*kKu<)Y#bH{ci3e!7rL!1@HycjW4Yv*`8-dQIh*TI1<0Fcxh*ecMp9;Y-q0hr zU4oqF(Az@FCt#b;+}jOpd7(j_4`I(?d>zN+;>+%d`?)Z7^5v6`tGN`&O$(OQnL&pZ zJe*JOr9bKDW*MQw__Vyq9Dg4fe1N&N2fd=7eu+*Z@p-1P`N#g)2JOSfwxl9Mjbq&y z&-#(cI+6un&xWs0K!!?1=R2OSt^gk1i{DJ$h(Kr62&1!(yh{&`F751Vc6IK~shcN$ z5h>$rQ=a!VG|$~devrQL-Shey9+?{{LO5&Rw&+=T)btc7DZ`()kte-Cr262G3-TUs7rfkn?GP_^dtoZYz3InRT$V z#Q*g8K-UkRDX~_wA0!{)CiI8LCIq^^KLHvTX>3@{9z_rPCqGOxX3NJRfShu)%nZs- ztust>%n1Wpzda2`(2KlY?Dh^Ee0cQ~2y&qp-n4@I&DgoTtpAl<*K)10ktxQYfo!UC zQ%!hsj$sb$VvofG&3n<^J-ObeIvOTLI~q)K%}ol)RwH>u--)&&rKZ1Qw~20V47NAi z5S7kRmrvf^jEk-EW3g3}Xrr!S%iMZ+(Sr@Yom-#o>TGQILbUOGW3V3m;=zU z4V&lIyUfl;mmTM}eidV@Hoi?8N6RvN{nJPKKKR`T>mM(sS>ZJvEAqV8dT$YH892x# zhB1%$MU6!+F^pB?>*f`Z+pMM#`yBb2@`!WPc$ZUG%`TV!*!|Z-BiBJI3!#~7p`8WD z9M>Roq}u#+5AxoxDcjGUzz5(qELgBX|6`0X_~A68>w}2~aU4!;?WQ$ztW6D*qeayv z)|3&%D*HGun>7}GmF2s0%2VFuv!n98c#$#||C2Z;^tB1zX7&;H{$pBTQ2b}3>@d`q z!0$okg3e3b1I@#0?fj0@ksatNk%V{H3a zH}cvFg3tetoyRPyHGA!vd#(O_Xp=h6M)n&yYGQN$uS%^#aQ8t&!1{1npldI5D8Ew0 z8}A`Mz+Q0qG@DY%SK`6&N+#!a_{jh(cY!BCw=(e z)jo70d3HDXJpPB+e%X|IKT4Zj0=$Xx7k*0l?eX&c)3HGzb4((C)@$Qew(4w+^0@8t zDX!C5?^hxZWU$^}LEh~v$-DhWWTf&Qb31hM`18>4I`X6zn_~w)#GigFc~U>X@7vR3 z2IT`imOQCB@jR)o?sxKTzlU5hz<1@NMmDGa8+kKjYox8P`?qstzcK3T?A$Hl)8FQK zlzF7*lirk_2^kAo(%yp3O;Fx3*^~4>C)L1SX$;=U^T@PItvx&|{||g;c_v#+#2pw+ zqpf$`B~~A{!(Ghv-D3h>oI#v5IXjw* z(5Ywf{#0mm$>-3MYaPFH*^-`frEkjscXuS^rGE>WZM^OH8QlH&oBzW&o|it^-@J%3 zZsjY!bSg3ucHJ7*a(jN*JT_xsGxl7MhU?}?hbjTqCCDM->C%0^#&^5m zpSKs4S)*#4zIy2E^R7|bzE59u26j%IpRAy-9k(RT!0xsA$pC$YmjpfVln?livDV19 zew^LcG28B-uNLx>`Z}F49!X!7f9Dz22=%pQSYPK;hx)3y-;>mznyGP{PrOg3CiSVk z%ITB(qrPY#u!6RhP>=d~7I9(U!k04x-qT!7Zddrx9n6_KnKQ~QT*I7Ue1~!i7Z1xV ztU1#OTqoofj_0aZZsGU7O#jF&oWY!7yzMy?A-AJs2c7wvPP>ISGgoQ%|8=fj{lW3M z+I+)F=4!LYs(t4YOSyZ5!>@zG`tz`>#rbD!MGD(oX;zH9HnG`8*Ian1>`&rL?cg^1 zo!F}Uo!EPsD`B_W;b+AAuO-g`Jdyl>m)m>^Tl1i2+{#w+sSS!p-Be}-v8(Ov{=Bg; z5a+YQ$4q>7dsw+)uCwiQVSN3_BXe)@HSk&Jp}5*yMm)>H;_J+1-Pk9Kzho|x|BIfR zab7DJ-?MFfH%4=Q29Lph*U@lev=Keaw~G0JeQ}NQRw7?%?TSpp&jsF5e2KB_Y-|$6 z^O--`-HKT+#K()_?Rm&3la9nTByjL2>c6!pFxZ~4+*-WZ!Nc3&Kt5?{@HlKvEy6|g zS#ZIeVApeDKX}-rcD~A5fuBz?>q8YbqzG#Ud!9=Tu3yx9)~j5>0X)^@v`n=5V!Th0 zu}viRa%d!w)r)G_Z{!=leB%ZG5`GVF^6w!%$3tIGR7b;s4@5;eL^-_ zkV72vNNm68?t63P+ey4ZxBhwFhc2KTtv^kov-{)y?d6;>`5}ZEXXP90;oP#5_8VF` zX}@P4@Ap}>qkaeQJ5#?)@TIl;ZI=lwe<7*;o7MibUI|U@!Ox%g{v|cUq7i@4yqLNY{gywy`W;5+QojxOxcXgBpVaRNHs3km z3ETJHHfT=gP~@sC{WhUR%0w%vFZq2S{=Pc?9$K~iqav3YU7~m88_B260^q~eGkAsX zO6xNA))$|n5kR^nIJ9FN^^#^#ZnUo0P3#6P6(>Dr!W@DdxYG-O5m{p@FD@n;#vV3cRY z#4%MNTWPO0uaL2!UF%GAOWI4Py^MH!9@_KZ$LAqefMONp*T%frt9(;}vwte{jkeJn zy08s5`=Gym8y;x#Dq~PNz`*Go`IPmSG5_QD8VBV7`fK-r zv9@5i5Xs66uxyU#bg$;MO%Hm}iE6Ml zc+p+U^Sw{)L5}X9h`dFvnhDj|i&-nYnA!aTQB=)!t(%f>(U3ly>le8nrz?C)j!gDdF}1d>uZ-rMF%U*jDct1r5^MG z&Bc4+&7%J+>4OcMaaNFhldcG7Qpr|YKn!>UJB9MW$PZ;IbByyJ2Wv*-^T7V6bdRcA zjNp_v>CbrhK=y>@J(TU|8}%it*0g$&#THP12kVew=W(u^r#2;1sz2BAPOyK79ZK;3 z{=5?F^RIL6(H{eYue!!IcfK&odj3~M*1EN`Eo>t9T5BfVyLQW@Ir3L?+ML`x=|-om z0^w-_V;f(a-`vNTEnaT@!hKUz{H6{0+N1NTu;uJR=2{-_OR>g?^-_IF;NkCYuztfY zJmj3zm{n@iO}_+EMSh(f z=&DifnKTEF6X1o#*o`&=%f0dTwLhM~DQ!Y;oTeWnE}B zr~NrSf1l^t+m#(=csb=wQ%<5pxkMjDGpeIsxTlT&%#4i%$G*NZ@|MeApfR{MTiLcd&U9OQ;V|ScA!#X)gNWrcWv8o~if+nZZl#eV`O{FIT8|CbEFmh&jkW z%nSJfN*AuVkMTmk-i9uE8McwF*s@n3m$fq&MKhnL&pDj^CH^Jd?f_>zh4T$;+#|4E zk3hFc!M5!V)ydZ*Bha-wv&5>wk8{i8hE)L0dR@j~G2ix4W*7W-rit!~bM_|HvNU2Yt*RGi9Kn{`;*<%~9I~ zyY?I9Lw4C28BZ*e)Akh3VZU9epL=mFCSqQU*@Tv59Ey z^b*r~vc71)$A!I2endONY#UG?9&oyW?+U)!j=fIL0e zP|V`j^xud-rS`Mzcj4JX@4URLVXbY#wxTskI*NSHJq9**)zda-s4Vp){Mh^*_yqvR ztH!zBpb0HCb{Z`;rA9DQ*DmRTnT&M_G`rdMWli*XIBW*(=MZnxPrC*&nKmwrf@lZ0 zI(C0g{M(+6wr7nj*;Zp^nh(d%#XfdFbHZ)XF7*KOwb-bzB|k13Blg$zHLkm(InYf# zG@=~P-@K{Bx`(xPh0Z|G9GQwO!F0;>0E5a1b`yJ_{4AS%u~qAtL;IoQ!^oPw$Rsu3 zO14?ywg)++=V)yGW^kxy4^Vy=<9_V^i|PCEb9RB-<~`a2);IOkk@)88e3MBV9^kHV z)j04bU|~Myy(SxQc4#6verc_2z%J}k#Ejj>UQA^QydjmgMnD7j25d3$!7;E!d7QD% zO`-$*K*7Z_@d(y)+h@Q-A6Jkwd&PH5OZ%23jA>rC3m#yEUO(MvQM($Gj;Zn=WsP!U zTLaI%z;Dk3=-SGNz9>2p&Y|BO{ou(%8S#>q7`%hETYM;)w&}lvYs&Qhnf%n)eqUf5FJU|{W?aiyuS&6XJA0SxFT7eBNFT~A`;(Bf&!GJU#jWVy@$<_Lk@;}d zDe-5DlhM0>hW{q;A9fqTFLcIMjpm!Pp|P|0IrVBEb7Ba~|0?M)XC|K0=iZR>-5kEt zoVoKkqvdnFKaHPg^wa25^=9E8EV_#9G%R?+W5!A7Gz^^{q@4td<$o+iq5cm}@ zfL7OuRyFo#L941yYff*1R?w0{YVQSg&_Tg)mLBOZ`x(`z2%+{1Y>QyFVLyQd(w zO8Zp6A$7V}E z#!yz}+Tf?Fglo;APnq*7^WL8vnwf3WP*x}m&HRA2zR&NUefvLr`w-K@{A@}8a;gii%;0iU;iT|A7l+G_rA)2fmT@o!cOMae#pS1RmG`8u`srnUW ze>33+BHvWy_;%6%gfEEBgvceHq6*p0=fRHzebd12!$>tc4G;34w{G6>_>RTTA){|` z&htEWzl*(pBWTZsq_sP-J~lBnnep|p##In4Pl>*`ANs7>Y|9MeJ*G8BygdwGtwJwi zO@910;Iu6k+n_R@`b~4-LnmEdmV(a&Y`|l$Q@{C-`1}_y?>?~&*+$=omd}XK!#u_z zvA%nd8N5A=J^vnaM6j5Bzg+N`i#=O=HO2#>-b&Lu&<{TIkmV9ybws@M@Q7=fTW|29dKDIJ#X{=zqUq zv>51)@_QH0u0r-Bcf55V&9u6q6JzEn!QJqb#m30s+x$ONbxv>>w9?1gKaW1;!ei|6 z(7tHsYd1MKan3|{&ecs`=M!tZ=w*W~BdCnyn|#kSXa0$Q_QCt%T^sVb7H@X=tjG3Y z{+eAbGjuBN#6wTEzH4nRV7*QFc~>*$lDov4GzZ8lvPyLdPJ=lhS?205;#1ButzUWI z@J72G&OXCG;wrAiL)+kS;-Lm-C3Hi_hZy^vJ51}(qGjqY1Sasgz8f3CTbm8Pc-e5C zl;B5)sCOfNW%BRQ`f<3!Xwm*@gtg35{W)7Ej!ZMj4_db%M($+Up={eh+vkGeG9&mo z%fY{Bwl^*ll`#I2&uo0<8^JSoIy_`MV|5q!#!l{E2ws2Bz2j@=jFIN+M7zJ)p_h0k zImQ?j7_8$P&p74^dnl5pz0j=hx8PN>m>2sH`{EUSIYG(A0dsY0J@wdfH@MJ$q8?=C z^(OV4td0;dYZZS#WvGt!VRa-m;oXdY=9(|S%oMghu^sylS5=Rci{*29MmOU3sf6f8l0*}{Iq0c;U9J(4sPuD^%zUQ7Ll;`Pf8)<%C1?@rX^(Byg4g->!<2ll34*mFtGC)ji3-_O+Ag#7_p zY)py=$dg^%&AHvY^Q1OTDacRtZ)EASlVpWU*nRU;ly*Qll{%!A}+h0es#EzxqFOdQGEGkG9TW6Z_1Cxw8!}P zJI&oW)SGoQwm~_HbUk4(hMsY}-?svbViQFFhoI5qca^-8{aNqU+xXI$IDNQ*cj|i@ zbffpbjHzDE4tUOa4wr!Ue$u~KWjT+@;pHW%L8E6=v<+E%73B?&F)IsNYAocutiiy( zcE))EFlX~@(=C_y?RlMVTCo+(?J{KieEhbabz6}&I)e{;*KVG{bA5T|20NzBv%vp4 zW9+D4JL};l=6(eFfaccg`~u~5mv2G=dm-U8)>zJVl5MYo_LeXXwa++y&$3;5!GV0V zB|}I@2_utv|9;=jyiuNQlCQGC#gEWk6L6P6Gg_+*i(E_e>tw#FtjX()Kx%LvYvK)* z2}ENvySQ%PH^DF#XiVz3?vB$&$1SP;0iFq`M{Y|&rcXt-KhbwvvV_(N>8*a|auM-* z#mtv_=Gu#{5-SR=t1sfQ4}%Z!Ze4$eYw1F|uIE~`2rjL|4RKz&h8#%P7s+p6+s|Xn zuXgH$w<0T_YtMlz;F~q*pk8=RWUBWm#mw~K(RyFS`%1O_ibi+KZ=~)UXk!uj*Y)UMX~7*|!xmCk z`ep~QO!&EWWg-t+?v1lO_`EIww~Hss_rnZjAft=~pA*g=`P6=78|Wg-gFS~cd+|eb zZD$S)+OkGqo+E2?pX1nS+6-h47dpGiQbF!M=tzCm+-<_Y@-A?09oA=|EnP;W-IceXOyRTGQJ9P+tH0zq*=Y>$=J!*H!^nw z-!9g3jZhl`#B3Z34%jWfiUe;tEa-w$>aTi`=_S3-S&o}p)?71t?e9NK$dl;_m(qkl7bs*cb zr?#;M+AODS=^$+nf6FfO#`X3)8;)G))Z)8@Uu77-O8H-g(~;Hd%cA`a$anbH1~_A| zq@kYY#QroqIJdunJ<0}mRDC+IXrz_sS2g|}uQ0zqS~1GTOV1Eq%+S-|r4PI$a3Q$V z_b_-*^qD-)&?0vIO!~c)Ig-?G*23AZ?Se-lBWi7|CSPw~e63X6cL}yzjjdrm@bkQm z*s7`YIr)7(w7(xcw`N4(@x)xX`{T;!w}6o`jN7iVhiq5yo46e+JM@RseQN5G|Q5%x8-Bz}~P!!Y>y$r|+~D^$TZgVHS;PHsm(04zTe`~hSs`tS(X zb<6?DkC{B%k1fgM|32zLKHi{b7jZ3lc@gC`FT4*J*A_RCuaL3Xk3C7eOZ^kh^7$$M zUfwgzm)DhdL+f&8=vL~;Mc&Z;907R`u+L$KD2P;{CbUa>3=g zQe#|D;~}|da$Fai_(0VQlG_@;Y*Ka)CPWF9#T-TMhjkRGBt`5P&G&YiH^uHe+XD4)Y(6)s*{a2sPryYAPLkAk4;d>mC zSK(21+|@zmLzb<_Weq?p3k&$(3yo@xJhU@lMHWjA)qX=(=$Sj2S6cU>Yiv-!2= zxnNxVaq8Z3{D=+;27ZdOImWk}mdu>fk#BC-vwGIiuEp@AhMPA^PnRwOZ+^_veo;_y z+Wq1I6)KHp?m4;TT5om=~(R01UY$w zGi{o<$)<@u=sH0Y^0N?)cZ@QMl%s2d`{rmPd8<~Ezp80!z~9I|Tmkg&$!+@vdt60L z>~rB0c}>%q_|X(nj&nVWP2(HQ#9qh_JXDtNnpk5zd7msheS|SvcxpoD@}wH$WY6#C zT6J3F*lD89TJC+omE6YC6!Ig~u|GAA^SoJ$y@C1#MW&~)5#BiQ?$3>Xl3ZJgzma{v z$2GA~yru$NpzEGj4V~|ScX^P3Z9a-0Q#ZZ^S@5v~aU3+J8bykC^H2wTvJ2RXBwu-l z!7=br8f~MScGoSa9oIDF<&R>sw1?gZ{2jK;AMn@cpMI2shjM74*VVl*f$MVl89-Y# zBOKgoA53k4;}iNja5Qc9Z>U#%-toJ}*3`~?VUN_qSvP_cS*#0O$#@$a0 z$U~7arRYt^_Z`Jw?R_xnp83bvs=FCe=~Uvi+RwOqhhbIS!WzeW*MoHnHq)-^rLTj) zPG8I53(k2S(aPW0{e1nm-?e#l0ksHG!7(>E&H6wmy^oIq~f$#_n|gfl z4)ATw$M|LB7+v;vM+%RXw=EQCz*v9>jk;hNycioKOZr<6t1Z_^E&EN1-TunWD7m@rE z?V{7BI@IwH>-)*kT~jmu15keJR77c&m;Q0^th;j0%J z!Qb)j-OGLUcx~pJhxoODL+K8}bBFtsIW|6x+@Wz5PTru5Ex*RcvzCKGsaTnlbK%gU(abC2eC{91o_7lRBkLq|=}iqhUd$f%<{ImWnw@ogX;u)*wa3t|c};W=V^mR+A1vouW2C;!kHsGIRWtU) z&UMI!V7lVAYM0dyZ0cVGI1?~l35+LePjd0UL<6+BI6(vS72Z3PcVkJskCUN+PwrPK zMwB%Hn}BG*V`guwR9-M>pb;6+FteLQ12^++hbyyL^1A`=cJNaQU6eo{&-{JGt!R!tNqk6g-ujkEC-qwN7~WX?%3;w4FU9cO6<( ziX2sft;JafS$}q<^OOG*@kBeyIjibx<gfFy7CbUyiObE2BR{&m-QYIi0*`&wGjQ zaAh>Vhp$~HIl}&c+`xY8!uh<%PqX zuW(JWbK)wGm~iUEJIl{?Vkm*r)izEihg84z&>Y-L-c-ImM_2q2Wt5{(vP&+{L`(nb z_%{y0=#2kg|NX@APweykv$4NBY3vmTy&0YOWU@-~xIc*8pfT52YVTOr$zwWyXiU8) zjA;a)>dzX}a^7o9^70d_0A#38BZp)dq5f{%~kKC)(rb=tWt?e9M-oIWl2rtft#|nqWxhUCDGE_IRmSiW^wS}MMXH@u#W8aZYS9+3Q(VhZx_`|%h zF4oMf5Q(X6Sx+*g-q(X?>+0Pj1n^DL%pi_2)XbAEgr!vx{jGGxCw1iz^H zen9fNt=CRl6I(?**7;|hNr1j|SD*FhqovdHVL8-Bs2!11H2COJJ zF2;Uac;1t(ibZ@2f8{@)|6r?u52kq1HtOjhFLQ)B@s=xd0A9baaR@eivw*Fa_C0y$ z2ODWuWx~X+oD9z7a$orr_-;wU`(bc?J$>Fu{ddK2UO5csJwrI32+mgx;rx^&oQJ`q zaJ~iDlX0#+bIG44gLMMqu%CGz74J*J?-#+Za@s~K-K?J}tfTnj-jDBH1M)C>i{?Z* zxuiVr2OdXH4bCUtt&n|y%ivLKnY#x53goA>^Kr@!dYxemlp*KmX>Ja|b7Br*+OzE{ zGshau6YG@A=0u!K8HN+&rG*=g#h}Bndy{eUCgl=z*p1Jk__Kv9?E_ZU!(a$Mq~@tc z@OEGwX^b4$Eq>1PFuutZz+A$5scZQ>Tky|((J<>OG7J8w#s;nP@_lSX{_wCrBz&)g zKUA1T^GbXS5AvOSpcA;&I%(s!s)#mZ6V48$fM1(Vs%OXLE@FMLbHrk%B|V+-Ca!-r z<&9uAI=5Z6r|8qm8cx~tMA_;h=840D`*|iGi{bv0yCln$(eD+eWw%%7_$2IHjI55$ zsT;qO8Q6lIwr35=hYp;`h{%WbcLx1x;JoKH$Csfdu0Lx1IzHCNeYz696W_#+cQ-if z0Cx@#aC{)!c?Pc6Kf|20dFSUG9U)=I`(N(&fK%l8!Qo|JA69lJ_p-~1{_DZHy}r;l zwO0oXq(9@-eIM_9|0+Z{Ze z%e%LL|6igfa%&4u#s8OhTJ--Do~|6$_lnQJQ`xYx#YuR2A9@*1Pv45;=^ftv5cnfn zGkll$*>k=|HVNVwmo=b|a$YyKV0_`?wlL<}V}^W3@{M2*eBjh0_}oFavR7+AOuVNY zes~u@*$BhrQ_DdYsA7&G`|Op><1rjRi)yZAhg4a8!~F)yI=t`N`rP*yG(Fn+4dlh8 zRpF{*QUdyk5Umw07%72g^CLcC(8#r3%i3I!+ z?6o5U*nP9fWzO>&`XN2Zn-U;)#hA64Gdk3<*;%F;?rvksJ7I&;qS|2k9q$kQSI3u8Zi8a*~=#vuRq2aQ|+wbPhj`p{D@`96$_it;WbvxOP0)8 z0sh-)4}AxnGku_%bvc)@!1lGFg*}D4@I7ir);z#^t2v=@Ub&=pP9@)I?a^Alf?PTJ zPIdOP20Vdm;-PFIecuYbLuY$6Uz+h>S|NKa*SXM6EppUu;*BLo?Wg`c=pd6i)FY&JWGx@l3$AYEkBzbAIDbJBcmuE7csZc ztj5<6-9A1*nPP*S`i$FI@Rmo&^8p=hUkWXXj#o2IrcKA`+w^@0I4z)@^q6(2rEUJCmPsCTd=* zo$2QHTi3F0F`aT5VZ*<|9J|y5JvOsHv_f_Q;Hw}%LK?VTTYH<$XX>z>yo=6KX{Ooy zLhH`E)R~@QY`B*CXHd7Bx;AGRv$gK1oigfN4evGdKbgD}%v;uk_0NK9jZuKT^4Lbk zoBy@oUi>VVYkjY?Z&$-dMlp`7SvRgASBC0PKFrT+JScN1b;XDo6Fy_Cd3ls=r|x3v zoZs(`p@sIj9!%{vZwJ9FiPwpkbQYRXttY}@|PRG8s zhWagN%X=2FIjm{MxVpIp>zXmHey-?rB=vb`xvX^j18<^EXAWG5PL_cVmmYcoo0k17 zZ~@QKLysr!+jQ>@H7D+?F0_3{9!}gF&V2*-u`)NllOAluUi9pA^z01g9eMHH?%*Wm z6K8MPJ`~62$K{l{jItGcGatJ$Iljx>wv2@xHkiSj(OT_g{uPqPeZm(5IvaMGE3@Lh ze&Xh*v!`s~zgalikvot}SwB5CZ+n^;@fho#kG06*!p%M@z6Sit_Pe6}tPf@631KgIP;?~uQRg;E@2UXbhrvgVJALVV{40JBE=4PP zx8xmU0m|xKEo;F`ytATdvlWLZnKkEa!{3kYK9Rgm-V|d{W2abc=liVCO8fgM_V*F; zzAQ;|`c+3Cj?gdJfNrHmK^Gg?YVds9#e;^~@WsJL;F<)LNF# zwP4lSr#v~54XX6d8duGnp26C-)x~^7W|3^7HClXr>j+2A)HBtgd}R7pIr-nITps^6 z4w!esK|k*wpq$p;NBK@`@8$3>wXJo>3(ri4ek$K+~alp zC(n}Orab?S|I{@6y^wK!fDcXbnNCOV{3UbAWKMlD20d1mwmo70N5;Yz-;uWtB8Oo+ zT3>EB_Yv;pONf1F!D8Z(khK;};&*5ZK9OV6tk=0F=czxk*5&sE>J}W?f=?>%Zsd97 zT61{}T4{qOBujVWyBR@d?%iib#S@56_jk*86B)Y%dAfIBNwnPUjvid=_D9#a{e#cC z{T*)icIACI1={Y&cX5rsE7#rME3@nCKC^pXH@={44X@9wYG|0-*6_yM_J%jPerRro zdr>sU#vfqSy%RVO^2^=!Y{5e~0qxd7)qMWI!kKT?vzuxO;s4+@A3d&W%7*>W6$( zdVlWT7@5I$n^Mq4#sFgqxJm(@6yQnms~8~mm8rB(G6GH!ZO0?>nX(GRN@mYd2*3X<0TjgiW<;R!D zzmI$QJ655ae860A{1lz|5beR7&N?AllCR<+{3j1VbN%e!NUm(+x`4G_@_z*V<67#s zpwD9N`@j+Ox5Ga3#5HM~dm4K`R@0+CfN0Qs5gxL+}Stc33ZgnG!S%-NZ ziN}wWApfC9E{O6hXYnO|?e~Mb_4094{DzKCKVB>@A!Do{5xB-7ezR@0w z&T|`=W3;sFGJ^iy$YtoDwFln(R>tTa+FF)o!E}UM-qOwhT&Qpxo7p1N@CmkpH-f7 zPer5r5(7V8P=435->5ds4D5SLw}_vyclJ~MJ>B11(DBH|Z{!)C4B4;}w2U6`q4DHd zL7UG6E()5=(W9-BA%zB^8zalYgV`k}HJ^kXgk z@R%7HmDmOI#x_pDHrT2@DON@Q`uVVnidKr}k@J)9i;-6)*XTUME0Aln{?M<5#b=|- zD9_n>$X3iF>=ap=->Rb+ec?U6u~Lnhx~`_Y@?qR@4)qkXrUcX$ZPz0cC%((OB(PEEg%qPhFfVMF>M7^tJIQGo)Ga$=BXxW7 zUT*c0>!*U8owrjr`|0FfMx_c#G4BcN+x6w-*2^=KbEB52_b606pZ>^ zCzcI-fg|!_1c-5!4duAL&s?=*!F~x3{xjbOK90>SVqP2kOy>Q5=CtNfB2SObmV9P^ z&K%%*%oj`zhSJavpS{Rxf@XdA78TNt&brw{oSX$s_CuedOXa-K+MAd!v4#}Z+EJ{v zqtS83pzGiRc0cFt;L{};ESm2MPDI{1T#Vkp^>6maRvoz1v6HW6eUlwE8(!KmB4Edb zRCE6e_L;h{*A%!D`%Dvtau?LIUN~|76GG*a#EY^+iG8MvxtA<#DDH@}A7r;Y06z{B zn-?Z-rfO=yQr_iu`6V7bJ2;gXEAYqz7oT6nwdPWPa@?r&>0|d_15U37w^xDVE0J?J&!xff<>QR&*(dZV z@cE!C!|78UIwSPDK=mfqS%>XIzUhu_OnaM7Eat(9&f5GobfR~8@wGdF>-X*V$8ina zWZ7%HY^BEWv#;vlhl#kHaJ2EqTgkHVHi9b_29=`J;H}$h-)pRG8S{F!FQ%UsJ4uJ5#MKtdWbmM+Win7+R+bP6NhD{IId}hRNG%pF>J}%)g>T z?D3*Qbk);tBkc?3a{9!Y?Zi`&&!Zd|I`DC}*Ei~2v)kC7h^HcVjAC^BcI*fAha70f zlGRWjeRA%RO*)VRJ*ELo8sm@Me;IhFVBF`k4m$RN;otj#K{$(6dMI~tyoqML*xm}z z>B{L(W13NUg9jBK(tvIei><4Wf8um<1A<>q3H+M2Bjjd}xUr?`I-mSOD=#TJ3_6Lu2d6Xc1mCZ25si`?@o?Jg zgMSTJvcWcir>*#Xc;Ndh!ISu*a@ZW4PAvJs*!n}~aOMDbG1|@vmJqYz_}otj{pr`S zRrl~MbD>E1hlc$6E&^_8tH}Pwb8fIc>6_p2O@MdUbNpi|o9M%LTLZp_IY&qP^}^Y1 zV3o|yS;j5O`3J4|6_cDCXO>Jm(Ms%1ar!C$Fm1N@qYcLgX|t8@5nK<^-+j#e;dQlv z`-D7mdWvM1lgDEhuvghjf7gSLmLul|XCa?G!~J;Lo5@*FE@Y2`tZVEC4C&+5tY^18 zqS!MhZZhL2x-a}aQf34fX*@EK)sPKJD0hHZ1no^r7M;Qxk}MA=;&UUTuP44Q_2OMb7CpD%4aaI?|!j@C^( z)@lm;_vGKQtKb2Ut&z%Pcm3sz2qV{xSFne^{EE8~hDC?4j;cf_=m*>~28rp&ux0 z@Y_%NbMp@*V1DeQA3b zx!-LBFF#DoTZ$tKF&9|>Kk0`fT>PuPII(PJT~EkJA8{|A=}(2R#Pz8#B9FYPyjG&& zzaeM#!-qusdGKmmF2na(b9vXWHviYY&kjAxnCzl_alqKyLEnlE*FX*bVPpfwE6Zc7 zTOi+wTAqP#Th>mt2gcCxo7 zaNBJuCuBjZqn{3sZ&KNEWGWpmSOF|vAmCJgt317lz&VQoSmVfB4o~z$M z`#&=z{;j^q&$|)cm8d5He~+>mG%amJWdH15;)X&2#@l=Yyk%G6u}T--pIzW9)9fb=giQB z@qQ-!Srb0BvbPQQIke|kp4*vY$i^BsJLd*^giY(T89e0V14__)fN}pbbL?NG?ebxC zb}{$A<@1S}{n@bJauV2o`hNiXVJ zxeV^V3fx~2>D3yLi7Boy)~IA}fEzjwdXtfiBQ#v%h#pvi~PWy*+=S&Y7eE@SIG>#x<=0OY^4;+BBlwu!X4gR;|9V>g?J|3D4! zNw2+4^*%gT`S*%q)9^3Gr_sdb##-`tf#Kw#tOouLWbAACCNfR(tg*fixlC)PU3WOe zuDcz+dM0%z=fp{zy{7(09!|0H$UBomon0TPyh8_LLp8Iu=*Ti!}^#JFlL~bbN@;%)SKd#W)~yf{y|f)~56H%|Z_#^5o}Z}AV+H{@8A z4wj4FtAFK?O7RAcATzz{A*Yn7y#DF~a$!H;Desgm`}0mQ3nlPa#hpnO!_mGa64xax zowq&tqTouy7&|@by!W0nwI+~5P~XrO&Ya+EMkDysRwo~1;=TOX-bi}?6W(L5%z}@6 z=qVXXp19&U5%$8kU+2ji5nsDc%+u9NTYc26_N3QigR*lTG*ef-%NXz2OpdRo%4x|M z1Ycj}jA5e8&Q%&C@&Pc%K6{@jeYXfj z_jR1h=Q#r!0!9ni5->FhmT9v%3&*ux`k@6ZAz%?48neV&06$^fG(yQaLOP)9#Mnx& zc&1??Y3MwAU6i%kJIxvY8`gYd2`$XN3aLpgj*uytYlwXBI{+ zQ#pv8T+Ldq?RgD0c7i*4h!8P7aU7|Fyj9zW7{9FLwa%ze<1!`aXkqc!JEt3 z<7<$mS*s-*)S;K1Ozu@bGO15KuiC5X0oRS_ls@W=vF6A4UVOwzb@n?9FZ?UBZXV?x z%j?D`o4U1El79nyKb|W{HV*lz*Rl7}NNmRwtV2Ki3-$}@T#c{(YMxyUZVYs1=^0<( zIsk4n$+0`ZWBa$S%j?$u;PH8`Id0>C@{)s(#CfZi>QSesO)Fg`YK@=o$Ya;$|1d?fN8_Qh$>LvE)3Q=I(oqEF&P{HqGQ)PMtU81JVO zlabh$J6WGT9`DT0PT)<3&jh~YeHy`ee7uZKd%W-&$;F=+IWgD;Ov(E)o<`~Jj{WZQ zH$&G+`Mxu+4p{Hdu7NJ5ar&EJVBTe3Z76nuIAafEbliSz>wA=ag0V@ww{7jTov@e7 zW^H<$`|-3V*pq#-)wlFi#}-nF-HP@WvX8Mhm-Vd=Ih-|qhsHrR`C5Fe)mLvGw2lqS z)fTYaC7efcNx<*m)?@2A6|8ZY(6{CSxE=gNxi+s|4%yY#Q~ThRvWLnSw1oDr2ItaG zG%o6g=%+P49`kMbIZ-}WxtWYZ1bo-IieE#vvC_|J-0|{((7wVuw(MJS-r;?+om!pE zJ8Iwj<-VrWm-p2^@ZLVdoPPwK%$X?I?V%I#N7W&kNz@a1`0*WkI@ay082!e+m0x+s zuBWN;Cp((Hw|3vw|Jk;0>%xA!9?2}VQ;nd9@yG>7)k|uk!dV;ZoDnVx*3-r(bbyA> z_DMcK&VKE!YyVyH5^J-y3t2$?iL*Ct`BQ!Z?f4|INBNlNl#;=N{BPLo;cV~~kLO%s z{4C|iB}Z~62ado?7h<>iIq^!tJ>b;(&K`2NdIlo+bV+u*9#~IhtPhu&!PstNxw8hl zIww%4_VF(O{!@%P`~;$gHJ}dZvRfCSGkjcZQ*3<_?F?By*kI(>(<`vY|f)Y zu4SJ2d;jt6$F(=m>zcAM;_`%m@8^dodw2%>0n}rC?9`LXx}X)oH;$ zBH!Lc{px40y*_>}^w=3j%g2nB==fV)Z<;FmWp>Crsw5iW+z0lGHXOV;VD-}W65fZ? zQqditUHZG9TyF=SHT}xB@EYrf@)#t_R(?v^h{oOY1`XzfWd7q!eHYoX( zb#4aV=dgA?!MHhmc|vG$(2luc--etVjInHdw$2$FVF{ivbtpH)fqn2H))(EUQs+;d z?~0UT&QHC&+@A|g;{RjTa&Ok^*8Vl7Re)UOY0nMzm!Su7&TwS0ZA*0G@v$xTf#(rFYg@rm!;oaq;fz3S^b)3!<4-_@rF-_UnjYpGX$rs4q_hZ@$kFz|fd zhKD_L#vzMq$q=F|o2P6sg8ya9tC^u$jJ@=fBlJl&vp%k4Tcjg;&=JuO?`L17K{_J& zTd)P@uyet!neVkX!yh65p$;k;=jBLm_)VSz3_Ce)ts(#S%HLH3Mzaaeb zYj(J;yU$b%Zw2trXO9D0_7-O?uVIY!t?VIMgFnsRN4hC}wPERD9%r2%7+i=ACp(v1 z>hQDIE;`?i;c@6FE7Ya_;EQD2ohg49dLCRH@JDacp6i5AHSNmhWhb^_=y5iRaFP6I z8u3>wqx=Tq59Bvg%|4rCF8Tb*X8mctG^u6i=V>K&UX1^yPS%l`;)$PDUTdOl-!(>K z3n#yi#@PMYXYXn)?n4)Xem2xR=;Rz?EnLuq9om~>4E`7xKZ4#5!K07BqYralN|f_b z201T9yt)P%Y$juG&&yHv{^`2BwhM!ssYCPA7-!5@y*AF+?^k(v!>jf_GIOT_n|uHC z5mxL%&gf-L*|XnRZtpL%S7b2;zP<~1@3#6IOnaZGf4a*uQXL!o)8y|!Hz~Rk7}0M( zls$C`{Ah<-0ZB%L{=uBO|@HRq#KKKiesn_qU>; zBpDQ#(`+HGF38TobFN_TyHUgdlx!9A$h#(*-yypE(p4)OXJ;U8nddfu~mbs zKKNfh-*wBb!~XQk=!bizp)-v`7s)g1wJ#U^CHxDMZCKIDFW?6d=Y^tS^4Or{t$Xcp zf3W|rhV=$?J8AKlpuUwKp?7@rf@B8`i5$cS6Asu@}+KkxY5F&Q7_ zR{Se_y(`E;T!8)XO>p#M`1kARnXBB9zjEZx1(HWqr56MylD_PoKSctOf8D(=<98Sp%d`Ca_dowPg*)sOkwv zmz)Be$$ll05wm&blvDp5zmiJuqd7a8chcXmKlrgHuB&LD94uX48+{p=E9wh^C0vU~ zJiX{dJ(9KC@69&0b0+Ar!ur*%KJKS;Urg-Hz>H>MQ8EVdkk_emeI@a=eXPMQ4>}k> z?Z3!>GcN`H%kM&9&^}QZzwfibeKB<+LoAdZoXNhO^pCT+_aHB8E$c&mPp(UKOP)}D zvI(fpGpUn3p;_{4*19cONpcmoPMv*f$HXMp(ZhN$ybj^bfG<3V|Ao#76x{XTvl<$& z*EFxOcB1z;hnW)}S8L(g=_@2rHgcD=MOx_p!8{<0Cl!Y$;t*-d-|KAQvhhxG3@ z`~z!U*6bD1eH^(jhclu4%WhJ)m0BNm|*0#3&IDy*EWIzmShQ;|kU+;)hCFCOPQrT*3cbFEuH84+1u9BJJZ{Cz2QT*y4w0BQYHxWPSc(r2{u-7r1Yc65Kn zBhkC(9FN&oat#&m9fG{TAHwy*q+b=82S*@b3Ng*xs?oY3i+r$ll!88aSMGNC2*z@ zc-#y;oaXpYkoH;dKz3^Y9ut8_5O|c(@AZS>0bg-7;j()X-l*@trSHGv0cS>!UC21H zf2#f;%`f+!!5-`MO|G!NdCx5IXM)qw4cyh-~9bRK$K^AjUn-((l>`2lzSd2;g%?{-^e zs5jrR_UwdX)(_)R)fpYKh!xi2ju(bS&)84`D zJVXCaoRUu#XbyMmKG;;Y;+-eTR#abZ^p|mOH8Aj;*gVXelL_82KZGNqb^7lF&N<_a z<;RYi_)x)yhjUhg@u7yXItC0*fHyjiJy&=^ol2jGA76~6J4w2u;K@U@;roC^WNgBbeQ~skTmw0ziq`Zm4Jx(*-or^vxh_24$jG@YVm9#n&Eu2uE zTFOIDIXh#X>5bzE=ZQxz9WmZrnQVAwy@x#egndQ5Jt;KtSY!cS-aMSI?q2p#?LhK- z#=M(e=Im6XeT85;h4barhp=OS7Ji4Z8jRy(65$4}4}k00;Q9n`eI+>3I5N$hk(uUR zXip0{fs0_W{|FDd%m%x|{X`5F_YwA;y6FS^`&RZC%n#X))yH7a7K4Eo7&!9J!(cx; zN5SBmvHs79_1{X5e!hMFh5dIj_tgIoaGnYb9LyELU@`bn+ZTmF(#j|dZvF%qXl@P0 z)mQo=uU-xO&!ho2;M@Z~OaLDiCE!B~{XP!P+y*Wz77XY=XW!i+&H{X1Fo2I2{$&#P z&1EwOdKkmS;6&%x3BV%FH!B8<8o@$-X6V~43m-Bqe2By1v!tVc9l{6NWZ}cE7(S$P zXWy<76WkTRK=&jk0fTPvpw;fQU=YWHJuy7^;wQjB^Y%XOtB&fqhI_7CW2~FN|1WvV zru`}2@Qm|6Mc8V-8{Ukb*{Gdp6fGyr(xJ0H?f90lZanEvFvoQEj`b5tI>ck~z@4K! zo3N|0KW(g&j`%2P+NjrPNcFr)KIs@0W|ak<`Y~aScsiByGc#I_YlO$7oPOeSbsrVH z1bUTMKV{XMVbyEt{0QeP<*VZexAeZyqBio#?x2V^C~O*h^;jP9iJGIbSy1@>@N_yK zCB2JuUJCmIy2)28{iJLLqx&(xPKmD9(!13?Eq*h@cMm;Qdg-n8R@_&K`?&o7Lw@DC zojji;M@IdVnc|n&Cm-r@bZx(?fU~PAn{j&!7|%p=xlX**KPgANmD;Uu$6t~4xLYq5 ze~*$0Jy7&T)o`MR21R@`FfZkf>}`_?f-Nwc1E zt}nY@>If`*bv3=J6iDHjX<$+QB== zrZF)aA#5<~(XHlVgNY7x6LER|F_z!@67VI1wha5fmTCT*GF{O!FOA`?#;c9?+y`z5 zSK|NQ;Qu(fy^q6}DHeRj`u+m`wone|xzRm*^gMVcxGQ-`IFtStcooJYl0P(h7w8hk zq|CI&WHtS&v88u6GS-v%{w?Lk^Tm1dhBV|G;{0jGm4BqX_SpHfM)phMy!qRNzg;@a zvxmKb?)?e8`4aKwhPTJs--z?(X`veYz^7XIkbWXyQFMyqH5c?y+u9rEz zzZF|y)=JKg!IPn5W`8;G!)v`E^vegYbA|6DU#poK+DW;0G?uNXd-+^gT@!u1Gd0Cs zmyzP;vW4)y^Gdv0=WAHwYbTkwEHct7KB3OfQRmT~Yo+^tE5_gNb;92VT4TB;);>Ql zY~Wrh2k_g)n=}4l?Uckt%y$_xDsfScCdw>8(!@!S{gjaYL{JU1}?+Ed0l z@$RfaZyCmL&kt6!A4|D42P4xbx{Y-Q`PUlsRmLyKw<~sTP-~F(vRF$vivi#!B53oOBzx<--Cud)|O`3I4nYzeUpC6@6mtE zW2B$U(?uSw$^S7=&G|grV(^9ThhJdLmOZo8mYR$U?3q`z#Pl`I-1qRU$a;;%0`gig zBJ6U)1fO1Lq4=KbiJOxUKUwjtW$S_SwS>oQQmz#2upf=z{kIN0nenudUdSWgRj0o0 zM7B*rN0H3F4!$#1reeEjXJ0C+tCeo01R5hdxKeD|WBNPrXEW%)g)q9A-AQp>Ui7T?FZj>pKAY$onZ+6@em<(d(^>5f+I|vyy5q=rKP9aJeoDT7 ziEkJF8!T9GUfsZcUAD8bUF#u_|0biOle^{|M$7!xF|Ph~P1qw7C%u>WC}{t5|3af9 z=Vo+3$VF|~on!%*7WRiU_pyuWR(kmC)#BIuw_&U9G_It5&y=>|2VQU|UcAC@i5E-yA#PI%U`elW|BZ^1*+G@cq-S-#Lwh;GGKfwleD%=%LU+;j=Q(kL47$a+M z1O52m_GHBBKScV(F&Rm?#vpEQao#^R=kDK*tY1w1gUhnT%6f;gG=?>tXDPDUoaP%# zo-dH+Rvw+@IlB_uVR&-OMnZQe*lk;#hH3dKD8yH)i**d$yswWlvBhVN{=zF(Zz#sL zX5!=Ad1#maG44VF4wfGX?e7jRv(C1Q-z*1LbxtRKK4&m3HtAO$X~5r)rK?HM)pI=K z5@WBonR69W|0~KjmPGF$l|TF-=f>JM!y6#$S!-ECrZs|cVs%i2*)dnHeOo_0AB`07v0+cxf8-p`OX4hKKwbpQv^`o91li@p)IEe4y5DZhvC z!M0UOR|vc+c`kLhI;wbD&E&d6Gj+y1DfB)t*=l|(dnaq8eEU0}0 ziypEb$hPt?+#?+Lq08Hc?*NDC+P93fwU-#IBhmX^;_$U#+QNM^;yv1uJbBRI3G~gu z`S+Y}o6C2xSM7pstz zYk6L#_CuoUzSi%5Fc-Mb#dkkAt9>=>|2ul0jq+r75vFxKFd6#{_8m@V;#U!RtZ>=n z1UT>e9P+GWxb#fdRRhQgtsmN#A4kRxb3aoqbGapZ$W7YkxdUI3jx@vC=V=%n-TUbp z1JAwL@GvjA3p&g52)HL*_ATkg%=%=b|6nTmEY8Vmk9t4f7M%um(iPr98}y&YKXUj! z(OCV9SFyecueWe{tnQ8rpW!~47TTC|*1o)kdC-#Cc@_H;Gqs=T;5lX+-@TsrGxT8< z?QOCi>++AU-71_npJ=J}Jgb-475C z^AOg64TNAF4_iZ+KZ(5v>Xn?QJ65OSBjFC}T!a6E7Skag37$IYlPO;(5jd|nk~2f|M*F$Kd&@WP&e)jeWqV|Irv2+MIAlmC1uPp!7-Px= zBlH2{#l!HR@$lJ%8|Z!t?52A`fANgb#5-~f&$#|a*0zo??s3ts8tNIWC()hEXYS_w zL34Bu-|?htK=?ZSaPbUiTKEBEDdAqcuUQ9&&uC$daP)r4W2{*Dy5#4pDfe7_X!wi) z!oC8pt#$p(-FD{P$KUar!Td#6@Y1gTVPAsxz35eCUrG6vZ(_9>xX+Z0rPtD~ zb=6;Iyj;M0JayLERj0#krML7I<3qCP{4C|H#Ky83d(6#RlhM_dqdVYyU{UR1*<%{s zTKl-w_+GW_F;|`$IYYX&Jj?EKyl+vg?h^2Hyk%3F=KC(^n>3!MDC-@{DhCf7X2&_3 zN>^sort&$~4B1p#@s>?xsby2C{;R+80T-C~&ILA{3cnRy^>2u4z+UOD75L>)I_h#D zt1vg#I?Sa5tbZ0>>Wq-meR>-@fQhc6noBf?WjFRwzdvX0#y3t`6mQy~RkGi#UKQmJ zv`+u0Ix?t3w5gDOne_jCc;sC8q(4%IeACFz;DqcBZoq#}%uh}9UTE-uwO9Ms{^168 z*%`hsQ$~X&s~Nuku$n1iPC!!>jvH7U3_Yz-(A6T}|ON@TcHx5WltZhq?4a*#(;+ z`F_j=zs$(U8Jo;;GYSpI>cR2ahZ%eqUfjQqVD)ivm$k%fFqs2ivUCnlL1at^_qIaA`ZqzZ2J0Ivek~u~8h*YOyBO(L zr=VYjPv%@yK4)z%;FHl+XUFubAK{bV%F7+^5nmNV51?^T+2WJ0h{+0@Xxokd1}EY^ zcFV`PiYh1IJ2v-|?|-bg*i}?jK)Orub%t-DWNgx|WXu@5`3~I`i478b*i1*BF>i3% zE!bI3A*~bpIMxvB{>PTTBnP%k)5&o7|{cb zC0#k?wUeiub|N>=ET`OE(0tY&>psTXVNoA2d9gBS_XXwFVN2y~;OtkzeUn#m-()%b zy6&62+2LBDJ2Jp+Tcbx0mskO@r>FxIb2}WI5Kx`8Bco z8$S*w#kVD%!w7I!iR?*AKhv^7>)mPU&^T-FKzxU2jAVh!V|5Fk^1vtjS-vbgnM9kD z#Bv=C4V-Tt)gLc6@CoH+{U6HBrA~|A1kQSkZ~P>l@fdb2+KbWp3@rA|WL`OQ4bRL+ z+AX^i^+|RoCGaL%=T~8SE}A4ev#WM)9~cLZ(rSzP-g7oQzOV3v??}+-JR-m2ed}AZ zv&_}m#dc%)=SDxf!Ju6$$nP++w*(((&91iNpA%nN6{fS&2~H*<=Tsu+D19((T{e2| zd4+~Exfpo|T$W7p@~>%6+^-fgB5_G|n%6;O#R~F53)e~~6|_62Xs>2Jv?QwI%Jh{3 zyPOa3b4B`G^gX%gd(@`|v`IGXz1V^s&&R)DRdj!+cPILAWO5Jj0eqPR(c7>eY4OTI ze5w_5CM7hvH5-2Fo&bKukW^4&QSDm5qdl;4`*oOdGy}J{^l;$x}+}o&yh{;Fi*AQTI|xw z*&_+D{uDg2Wmi3Nav#2kwbnT3qt1zm-b>#g+qWtI%bp;6H39wO%VO0q*Vf=PCI)_4s)Bu8@XtI+t?^?G>`Q`ypa_TZzMls zZ*#Wf*4X(Ht%HMo12`_`d`Dw&z{&lP>c8|!jqr4id< zQ&X<#sXFUktu)E8G>YrRu2F4FhV~AQGl?5KcE{nX<;%(PPlP?5#;{@-W0>j+pJ43M zOk36`jiH$^hTGs#;$v76A4A4tF=MFqCQ;rL@T8EjsWb}*rq~BwOCwBo&^f82R`}bv z)WVZR;7JX5(Mp-DA?MEOHxh2)2IbD7+_*j|(GLgw_Dcq5SDT-R@v^Kvsxw};XmxZg zbA_LTKP*>Vu2Cd9!MI((_p2SM@1B>i`53frYmJWQ4ZdUXdY>pahz|aj)!y^xDU%qZ zTJ-;)Bx@+o73BFB^Hj9$ZTj;^o=@^gAeyOll>UEUxURZ@{sr*4ul5R-(!K0@!Wgj9 zwOx>|NN|Erw|FdreGcO@PHX==hjDlS-FPkg6A}3Y1TL%{FXb8BM}!Rj1hlG_eGbRy zOn1gS!yAyz@(66L)Ia82txfjJ=g<4b&-DwY?UdtKlyi;;b}-M*`62mD)YsMg5A#%YK)+uJh4}epEgd4y*%$gJ{=%v`PN8SR1xc&R(^5DSapy?Qk2M zZ)3mHT65HgcJ`+%ylTtw{Oj@PcnB847wzwuhaT?H{!Ge{7#`4PC;v+y#dhsc!)vuA zVI32$>Fh}}JV-?p=3z$(`f<$#+BR73)OzsDOTzc)J%#kuqGMIU2l^d97ap-2-rQVh z_h?ttW(7S-0qtPDUhYq8fmbYren#_(k0@S+tvqEE^ZZ*s3bA{|4= zi*rWfzBIy$J3;uab8(JXdC|C1)`DfksULwdBRrPATm5!K?fZj-1NZ)8_Qnkv@wnV* zo&H$dN#e%E--;aooWSZefni*D~ zbUrdPb)GjgHI3&+o^+lYcrtkA@{IL{w#?z3$-9d8INp`Kvv^nV&gNatd%U*~o6i$d zCwPzFbP>AZPocZY@t(1p;WO)9;nTy-a4^{n2S!qN4v*alJ^f(cDs+v(r{Nzs`|HXu z7M#bK3BK>szU4d?&ipWj7Z%P;@I6krU>l8#;RZN^vy^pTJr^Fs3kzo^_#PyDEl=QH zxm@jRa*KJuh3j0|D*KOfbb+&N$Zd>@wy|(b_2W;WsN}GIAh9gg_?de57v7zgo z6tn;5yYqEpTQ>a&q?K5-`Wn8qb_KB2*Sc`*hc*^KE2F;j8fVAL|IUxuyQRF7Pn)datXZ4E{~v<;TbL7HEc2fEu^B%83jeS1 zb{pYWIiD%}YS~Jlciopxo}VJ$+_kNIG-l4$hz_mxfQL zm4*Yal!Sw?mEa51(VfK}U6^tMn@hqu_UL--x2K|y%=T?&ojCTx5`4<0xyRpPc*mjR z9H33d$$xLD;dvDwWjXAX?AT%~IP)ZPMQwkBvbNLKon|;pTL$dOo)bIAc}_RkJ&|tW zeotJF8J>m>@&aHHxP)`We`U_UOZWlaQ*Q7Enyx7d956aU6HC3pACK_%Z61mKk$&wQ z;i>+);943!HM}%@hW=>nYr7qpxw&grUh|GwZNsnLn%lf{R-65~S%uBla8E+kfc9_< z(j4z83GYI0(~2&-0lu=8w%2eLsFgDj2LFL2W|(^>!e{Eta1--t1CQ#m@}o=k(~n-- z@ub3g%EHNvZSeB4aNo4DaFD!xN!?Rf&+p>i zsqJU6sdtt<)Uw1_@O$=N4(I)D!x?jeZ|<ID5pt)J-WX0^P2Tz`~Rjrw)Mk zr)}ZdB*tXPxS~LJNm%Cugya6Q5gvH7XY!0oJU<6l1HUf`JFjh+-^N)p!NbP)et4O7 zc(njy`)zpipgpBKMEyZv@;7XSAEf@q7!TaJW2h&Kxe}-<3v)L@e{Mgpf1oUCzp}C{ zJY4;1E(^CyFYsg;e4oQ@VvBepo$#k)>7FeMpVj}yvT(uS$Toj#S=fm`Wb+Z{#)E|G z{-IUu-zCAnXwQ1$BxFuxgrpG-^K<9m1FjC%uw@5-?DV$ae=9JRe9g+2+#OiR_e$&A9hhl^`>K@R zwN^aAbnfsy4Zmp^oI8nN_gGgFwpF+~tnp=z9oWhF&sQVsgWzi*HaEDRrXTz31$`q4 zJIq*w!0n^p=}D9JaaYF|%;mZ0IdUqZFfg>Ac0GP}(PgklWcla#pz(#h+Mbt|wY&4Rzift23BZz9Q#&qko$DY<6HLwgws87s%Lui8Nm$ zO&qSpw;27xmt0^~yh!Jl$8$yy9UpkmcS97{nrTzt7#cIHoX^dms*yqDB0;+v8*I%3VVVbj=w+H^xpf%)^{mvSDiu(}r?2z3S zJ_Ifw8V;|J%mYq~FFBY7KLAg1Y>8t<;OmYRp`KCrHW_Kr{wsGuw?ZZ1V~0zaORSl! zEuEhRH}S9CYMT(+0j(ZKUoE=1Gu6{5+xFbao>I}f?ImGHMw&Z)x#7)t1bI&Mawlt{ z#uj?TJ(6MP{m776?8z@m^IUp+<%*}Ed%0caUiRwOGLN?1#XSVA_7S1?>BpPcU~k0+ z+vvoX0&B&a*kZpMv&A0qmk1{Rx{ou(S9>qI66~%E{~z;M_K4Iax`ymJe~(y-DL><)gEa3YT9xL-PVh#)dTO6CYL-p zjG6B0(|G4mmo@I}{~SiA6}TL_lN_ZRrRWZSktRCr3ZG%VmgKWeCBXZiTj5RyMPz2u~y4>*(E8UbXk^S^Uk!zSZ_H|I^V?STtj^(Gg@Ujko6|SHGrRF<#TZj6HwoRB-c+MSTm)R}|zq zrldmGUd*cRX$MDh^0=>;_CVXRcTYrTW;&CGJnwp_&yIa7_Bn^`rQyKH)GCMbqA}88 zd}h+F4QId`E3fcdYXWvXo!F9TPWM7fH;w{V+5fL+Zscn!qp|a`rHA~$s}h`B$^DZ-?qpM0c5H0w;S>GUE_26OZJy(|Vd|5Lo>#x|x=$^zr9+g)HZrA4<{rO*^ zyrYk-1s46TYW5DG|5ecc+7ic8^DqB*`zCS70t*1a3@SMY|;B`|mU?BlyFII+%G zF(0aK#Sc4g@ttnIk1bt#)yiaJuY)@(3xR`G7H~RD8o^1r=EFB&Z)J0A+5csOJ60X; zAUd^Tb5y97|1*rE@a{0{O6`%zwh!Q6-iMbt!@NHMz6-bC9gN$qNuKbJ*b@N{qjYjC zwkXWk!YHlW6{D3Z_YiXD`^6RB56U<@O21Y@7uBC&ohzKryxwW!93kz|o`T-WqqXH2 zJcrWgUQW?~F4h;(m%wIMN1AEOA4XZ4+nT$dWo~TbKC@>(EbpnWGMD?`jV!>R9QziR zRc@R{#A&){ixYiiB3)NGns=X~Eb&lhm=jNpG1iHvxGl#R^My!x&#jRgd&>A9U~M=B zY;NJaVzp$dYmEi(=o}6BA4HZ*;w*cW`4woXBUF1u%ZBpCoPokd_#$Yg;Hmjqk%V0h zG$gvGINtYd>T=C9wu#Qm#^QSF3B>sF0DSph3Dy-3~f3}o8;%uOP}L)9+keA`d2>cp#BLy>1_|d zzkK(@57&ctIy?Vr6YGCz=Oy@e(H&vqeK$6s0|1Ydeh2kVAUqKl)K&{Fth>?$caY|F z&L*^f3VyI8oBgNp@P|&{z_ZZb_3-l>&G6Ayo+elLS<=z=edD)xaH#OV#wsbJI;HhNm(=pjvvOrr?*Q`xVx;t>} z*0!ddv;1!~%=R{2GZ0*9TKrE9wzUD|YVLg@FZvXCDao2``1sm{e=q+VowM7Tnr8)8 z(zm7|1CpZ`X}zT0mw!r~6M2yj#B*X4mL z{6BBu$~d3;r#aaPe_-K=f3oKXgl&T7PSBZ0+Q( zoSH}BGm6m#*i7!hz^4Oz%zotm!;R<)-X(tRQ?ndLT?4iEqZ1y!U~A12!?E}O0&n~n zqa*Gu#*e}%aGvxlY`Qzs)ls|HUc|cZT{DSfbCcChSkFKUjgzmN2SUnBduc{iqb`bR}|pb5InXYzhK!V@A~XP;$bESp)< zzm)gZXlM>+@SuTD5dSxx3v|lqV?2K+ESYf%++>8u$Mvl@j`947aOqmlpchS#hkq%} z^CsaDoO}l4x9(ih325@Z~vwIeI*7WbUKewTGEc)DT@CP~}c<;Jr z#6Vlq9_D+urGsm0`YqqT<9imkGRjj!xdF=6U7{t-9hKX5&ttO^$MbF4Bw5H$d76`7 zei-{h!iM1Q6j_}yp5O~Ku?B#5|KuI9=iyiOlkWx%Y%w~H-pjeOGVvbcxl?GICrEp} zM}`fkFaDFSch25@&s@TwUx&#z zeC5^`Mn1gNzWlLSw&l&U%;in9fY>bm;m+CKBj_EE*arOnBHzC{XKVg>k8qzZw-NcD zZsd+Cd>%^19Si-^y7W21Ma$1}-e2qT8Tghgz8@z1e#XNu z*=!;HT;Q3ZQTv=mnmZZU`5-tg+Y`6b6_(vVJ!fKVtXpeRCG$FbBaBpcr-@&Tp$)TB zS;O8ms|U8SXIO8-JHt<;63@R!Yv3Wu+H!^At>T}3MaoWscS;U5+!$H#FWRYf%|uV$ z_9*&D(%(iqKY-t{+RHru3}M#yOjr0azN=~P#MEeelba2Axp6*cW;VQDvTqCW?~jl- z>X8AlCtdI(ba!p3z@L4C_i1;keT2IW|I%&vwi-dZf86+$fj1k~?vYlzM-b1y)$aEx z_qWi9Ekm`R)it|~zE%Bk;)i4WqU z@wr-jH+^5#s4*Sl`7>=1jPdm`GXPJbGYoNlqk%Sml{y3~3txFR@qdnn^Z$U#frfum zZ)`rA>Wy_*P~T^G{;lmmbIM-XTokr_*{gF3(m^?}14`6SrO^Jke(H|1B|T3e+k84B z)%_-P)=#m-6`Ue@Ndf=ON6ogcWcvh&Nmm;4ENkT#PFQV zALdC8tn8U$Cc9IRN!(^~_vbPkZqX!UbzdEF9`^w)xDDD>4h`2no#c>m+AVwE{pcLa z83To{;@bt?FBssTxF&QV+6yW;#uy>1IC7mE1HVDvYM0K!2p|8AE1ZFxvIo2p+yb;^ z7U|=0-yUFZg)sRN(i!ao;9xHMiYMSRk57dsy_|i%Y3zH=gipQN44*EBe}-p00S`R@ zFMVc7QjyM$9A^&6=BAgj_VZoS4L@+8v`BiKw&T)M**(Z-YXv|5@L^l`bAMRF6W8D$ zHEC;ut$UWgx`F>v{vBHzo67K$>g0|8)Ut-z`-czTD!$QSK6t?y3+{39RQ-^7Q+I>+ zbTi|#q`WBSdNbkx`doknZ@V)tnvCg9N4blJ4@vr$5&m;WT7+q<}*=fH2 zW_Y*bRzxS z-LJpmUp7^f7?Wd}u8!z=W?&RM)67|AV9_wUSM4C5mww3Z-bJ0qksS>1T==Z>^JP5$ zAdUR&{5+cOcL8S*a`9JL&bfvx z%5w}sCy8vL_7?EXn$>^8-m;-x_HwDE7VUnW?{Uqetb2d*3aon#nNR2u4h#tg{xu{V z2@MHfMp?Oe#&q>Zx|FfNE(3Wii?e+4nIT=$+rx~}1EhV|mR8r9BYYa`QyXfqMU|}e z7T=@=USoZu_->O9h4${grFFjd@7wP|X03A}9|RNrC5N2K1ooV1l^l{uoe^6`X#a5b z=nh;%JI8hBK);(g)2VRz;|%DwowgGS%`{BZVs^NrDK^!V#`R^ z*}yc`K=dq&!suX{&Dg%xM9)Ncf?orx;(T+top!2SPUCCeEu>w2v}-l(N}^rbV^3_C zm4H)@3`u015kFKp4o#MWpwf%ke>VAxl z_a)@9ms!u-SjP{Wj_y&|>0H5M;8X0!=BJ3yA>JU~2yf+H(>K}QC;@jQH~E24<>)l` zRPxu6cj{1cz#1EPooqAP9moMjBL`khN@K-!jnKGb8TrgvNV561FJ;$ETtdlw@Q(Kai5YUBJXXjlD< zL&7Im_f7T-^Ky-C@;P&QxQqR?i#!hT1HhyZ9!mV@oXbkRU&_KyLNDi~wbuzupDOBV z-)c^9zi5td*CA7H=Z=w1&lgV>0)NrRljta`nZugv;D0)HTQ9lZxHg=Vj?VuPV?l0~ zu?^ntr7N>avA@Z*cv1&#C?KuE3m1*@BvnFxF2=S1n6fAFQl**g(HOhfue1J3%EHy0 zi^>CD1sX^4YR!wvc)1oY@Zv+<6Fb~UkEYL$=7-0%{+C{3mTnt4x^zU}kjz-_Fb%(31F z6ldRh6#Tq*$4pN;Ygyoc%R3gm34FB2=$vWc_aA>l9jE77I`$o`JAow+uaLf7I_wtk z+`&G-cz@O-Yp=h7KI&icRa}O8aZ>umH=#%G;&))M3{^#39dY%oQW(Ky?aaN zh6bw3;LS=o(_exP`!{#!?1AcYq#E-i&m5q?vTw51BKAOKyQaON%z+QruVe=Ol#dx z>NOTz#h3+$a|advM@ya;e?j`$rR+gCL#6G^v8mh>p|GOT@HAw~MM+UuCB45Ytb!X7qh z9nh8~*i_|onf>{^l_w2biJG6l(||w9*GN8>+nCX^!}P#YYzvZ4c3PWP!-pBpP%ZBQ z_&q1*`U=# zekgqRq|(Ad&VctJe^fRCvx5=tD|3c6^A?{lr!lIxyp%Sgdl*wb%a|_MZsu+b+3-9y z)YY*ayNTSq>rdGUn+v`cnyKA$%v3kFgYI|lM2>de{7@O|$~O8`Ys2Q`=kR-(t=3sj z=GpuTYzBhhLxmHYFUIed%UmmN?sW|uy@~r(h!1k#RiMsRG|3%D&QL(hOY)*=yt}^+82(MWqva%WA3Nzd6Xkq zXbqE|byF$-(1lv$F!|743|)8--~ZAtwBT38`EvE^4s3zGeSTH$|`S1$vd-6E)oH5wUv|w|TzTTz2Pn{RufBC#{+fMu5Ff^s>bYy+IectP; zZxi)-H!o+ea)`Hg^1KURw23xHMvms}A#*5gG`5V?vB=@c!7eM9Gddi}A074+X3=-| zHQo<~!s|?(Z{6KnAla8SsDd>Jnq3!$erWyOfBF3Ifob!@1s9hV?&gf|vIe7nC-@_p z(agK5!SKkZhSFPQ*El>!(aY*yE)V@q#FbujL6(l!!j+jG{|k}z;?t$;Ea3hNi-u#< zqdk~Apfjx&otbm0jIir1I+M~}Wu~~X6CC4!uiXx_FvBbH2!bk|0e!3 z`5({!O#ZLtKZpN({)_o9TQytqL%V(SfK^wn)6)x_^IU137Z}6MJYVJxxTB1h(n-Em zy$vUj?JS(S+!MzKGY?weYMB2SU>6@hLwA`Teqwg-(I;jbPWGWrJ~2DE`5M-I8~fAj zFC3jqpU$7LGvR5^!5Msg0yxVyw>CxxOPDLVud#snAv`MjKg^G$LGxo6|GJ}6V}BGF z55~n0k4NJ*{z3X$U`FZ1CSZQ9ZSaNFv~6(vFGxfCzet)m+{_2z5q^Jt6z`!2q74VA zE0IRM%YWdFstaIpL7C9063Rs84nNC2X5f`UX|dxw*G6dh4ASP(&z2bcOyKu_T}OTv zYg?SIydHzgN9s(33w35C^ci~)i?4}HW=>CK9#7-BobQ>uuVzkL^NF+K@QdQ*PYq#S z)Bh6AK@XnSZS-yb_NwrKomJ=ZErAXMNiP`7j-dg5+<*92;M=ce`=5StcJH%Kf^+zxWUF|0MtJCuaxH*IB$UZOdbv_p{C&;Mv#@+@aa=@zl7) z=gfN#Mb{UtI>%k19d7V`5wvSs!{{X$A79ZSp zsizkFa$IbN@1|{*ytQMxrv~4aw^CjWb-Yga0~4{;L!P$e#8`hND;ATcek}XqUpE%? zphs{JhFrSu_$7{wmh6XY`W@uZMEUWvTsEV}Mv?e)SD=JEy(K8M3Zo%7E z?d*nPn_Eb`<`CK1vrf5qaPveM!E?d%iEOjwcfO;=dF=pFXHx zB@6kcPmFVVbqRV<*6r%+fgARePGl&@%dUYCddu$HeMhEl*?q*_9ylUXXa2^`VdCVj_q^x2X{mY!9^^I!5&S)}Zes4U{({qJNE`=qEWvh(5#Wsws86J?QW zsORtCsx^Lj$Oc(vKm2UE!;H!ay(t%tf$jSLJO+~z#-KAd3ilx6P|P?qRMIZ+qKkLD zjzxSYj&l?E^qUq<#scK{fsr=mhe7pGu<_> za1XHb1JohezlpRBXNJ#^?*YmG(9>-d?3*8eXVo4R`N#$Uc_Ys!@`lVzU8Fv z999zUN-7C|nLM3~9iBG``!jp3ofBhw39g3u*AN%Xzs9?Jm>J%c#6vp&@8L6gemUD1 zYHaoYiMO+1w)SNF$eV5GRs{>SFJOOuLoWZHL-%kBT$?eR^9Emad8;`e9B3ZmmF{1A z#y3SudRE#p-40~b+S?o(4d(MWGb?1f;`t5IU+2;W$HT5?*<)SqU|$toc>l>A8#ml) zPwQ4$+%dR7c>}jb^JXXH9i5Q3oV>jUuJLmI@l^+Ha*`)_t8-%^x>cRk7C&M$o$i?@ znS;`C$OkHWh5r{lahy-HYz7#&xNdmv4$iDo*HQGt_<`#$K)){CtaP+}=!sd=mj`H9 z!zbuy(IteXZ;tYO7r~2VTDn?m+`$R$uUttTasBf|;Q9Zj{K4h+F?Xa>`A@pmpB=$Q zAc;Ni`H=;yXyfjAk!>w@F5&x6kIhb;H;eumSuhKD^g{Ck zw11^!Drkm5IdM94ExK<1f#H^pbP;DD#A8WET6Q2xC+)mHjxV{2e7XDaLpw-E+T6v*={wyP}ilb)=tC`~H`Dl2}{*H?pmfC%(4k z|8KOez~3Varr(Gzl(%#Y(^z8@%Sfe+&c9Ixdf|eHcWsP__WX_ZIZQWx#(Kn4?)dv`@ai4y{XiD z6MS1DUsCf1_(ea&W1K;5v)bI2OPhf!{rVGaj`!6hjryv!OzZ4X(yPq>ipni7mDc20 zb9N?q#5*ai)_0A8c!VZ+p!iyHPOg9#{|9T*d(uB(mW zH>O;{U9QLt%Nmn>g~$-0$DiD{rMCS@-IVSlTPgxa@RztJv})D9x;;NU5*pfhq*DL1 z1OM>Y8yo%ind>%aO;&k*tRt#dx|Ggn|DP$X$@S#YcIjOoxEI{ThIJq5zriDWl{{!b zV_eV7I0TFc=1aQQ8)9Mccljrj*EFcSN3H*;Joz+j5l&3uEnSh|Al^YdPh7uy z59wtW6pshK7wC=h82|G)In%TE|3&RpZ;Q2OJ!gqdBAfe>&F@ux$mVe;8d$8D)NlvIUDcfAM+h|AY7j#^nX-9?WA*B|eeIi1QZ>*m;V-Xh_1&A2`3o z98$UBFCx$!{H!l{&4P8*UPA2{8*A4aPUF@1*v98qa{|0_E%>DQ-iVbMr#}xV4fsDU z7XL~tUTc}^j>CIYEPhKY{&oCnCeoi{A6&SfV8VBuALola3xTQdLj1hS{ZD$cD$>X6 z?x#+{V-4@CfwlHKtTGbT?akr?*dwh)KjTl>BYl&0Deoh(cAsWWydfB}N45xC8toDM ziZsXA*E!C<&a3Rn$Ofzh-|xzIH~GHPbfMjd!MZ)zZe$x}e4O3Li-gN=#QEMsJ$b3( zxv)o|&h}Wl7E*q7iqT&S4*|ay*50pleO~GC-;L_<*8ofJ!u9iaS4aOZWe!#RrFDMA zpI*C1a%uy5V(b6pYxhXMS@8q*<|ek?<23On3%_OU2T<0Kyo!h7djms8S3J~+4RAI4 z0A;*CqFHO|B15{{Y!wY<)3nB>Fe9szpA0HA@(bk#w6}FV5zb< zqjOi;OHcggvR&*~IOZjk{rM8BY~;lY%5J0V+QO1TCv7d~se~_a<&~c*I~w&{S?lF4 zm(us7_qJU5Tv_E0?VxYtfv;qabn?ir(=7B6!3T^EeJe~h zmrA2EqfYi|*tc1pe=WQ$GC_!P>R7MmOlsZG$6e`tWk$H{^NPdY2sm|crV*BHRrT$h zze4UWzQ|m@3Vn&u*g4C@mM{-n!YSDGRrZ>!y)m1;v!y-B@Knpp)f+6G9h!`xh@r-62(SG#pT9ybb%dJ+!ec% zHqXRfDq>3x)$$#lYM>{<-UPVa2md`bue5NyIoAE`7^hqMk5^e!WOFB3t!3FaU*CaD z?kBCzrs%xw5MYDdnsxr}V(#~DK;C-ZmfwD>>h8U=wfmo3vYYv8T`z$VOM~chUaDws+C? zD`|7P{pJDj2*ud-OPA7Sx9!!MVyFE>kT(kNG}g5@-#nncDV^FRxl_Dc!Nrb^nM*mJ z?&O{X&Kw99o78^f;zslsuhl*14Z|0`j(^WW+Kw-kj(X;BFZJxE3 zm_C4vc!sf)?Pd_X1&{lif{_Ic?DGr%bzhj~iQ4mLXuo6lILrT2{EYSI87t0}T6_HP zGVq(sJ!Fmj_~HzGTCO{;i=U6$?+TY& zY}Fq-);BTMy~#R@$a+d1^{)XR4f+mdqEAFG^*pwW?af~t(3ojYNd0SDV%xh&eq$Vw%ZpDV0Vu%m+08)9^!&R$Hvgq67Ph|3~v_u&Pk&U-M#4v(qC_L=0Lo@UBIRI zd2@LHTl`$;L9xeN?qJ*kBcW-u!6Bbi*hMS-&q;d*T>hl~SV zRYSnM;xgyPklhu&3OI*`FmzJXGoBEul*LzgWNn2WGpoHR-2gP*4TGud#?Cz#-91z$6e-*45xbn zzA^gfPZDha79PuY<2>Y!4QoZ$W2ZcE1Zxd#o6Ha&%68U&j`ZJ-a4& zT;16fM2^;eLVW(lX}J2HHrnaaywse2y*KjeXxfx#IJ$$Bk%Uc~F)3+dpw0;2d>1}@ zRDKn@FX`wQ$6m@`Ot}`_TVV9Rz&?c^+VKJVe?uuJ9=@1xjhFN;g|*I&bC$>tO15=w z=MZo8u?=52Zl@8Rvk1D#(@Oab@8YK@gL8dO^ycs_)>&C_$Hn;H`H77$6?-rUbE=z z%3I+HZ-F$KlF{Y9oHs6VEWC>@SfOWh-ahc*- z!8!rgKWR)>Dc#46Ni*ROo`$XsS|1-FT=x$xmhTZ)k_8tNKIaPHl8ash2R6RxqSZ<1 zv2~UE#DUG-r??z9BzTqW zTAduVQCMw{`qHY{ zOvwY5Use1ntf8#9Z-DHxH}30?zo>9+7zl}V@Gxs^cH_f*gysV(T@^g62osrBM+M~U#&6G822@=HuXW*D)DImPc#qNak;`;?9p|-?`dpVVr`_YZt=1f zkJQ6G%cL88$8P)%mjd8dt2K-EYVV{Gdt&(GZB6*iS6JMqz3$DOMf(!}Rj%$dP@cww zJfA(N`ORIw%ApOx4DMFuJc12)B-WcvJA#z2 zy7Pv{bkOh+!s)a@`9{Xt@(bFMcu$V-#v5FH`=^u<8)s` z^sWJo%VXfX@Ja1^7I{_ckzgZuXuqW~B0XY;?-AN}>0RgV8Rtx|?>XXXXmc*%wT3NZ zhfdWSW@s0EuIGOhZ>M2%=U}@;8_*Rw@Pp#?&B4Z{)x5Y48o59`dY$$F;fX}U1dE0g zb8a5|ow+9Ou0%bEK^bvf$P{O#y98ccvbVo^l|`eH47+>kp@-Mjl}P3=`1v8-e}cw7T>7 z>`LxDWL;Rqnk{)X$#4}_BjYbZ78GvGV4Vwsca;uk*eYXrJ8N{G-8QAVz*z1CzFG^l zh9z^)X+8N|#NRR0oVR90_Lwz=#^^h{jCu9%8Oz;`Pi5m@qHZx`cj?1>*q=-ZnbvpG z##S?V%L}Z>jb^g$;kJBsj-%eeW!Heabt;c?CZgMom(zvr=F*3wn5@q3eN;Sn8OmqqVP;Z8CKb2AdV)60@O4V@D(Jb%68{9N3G&y5sd z)k?X!_?Lk8?{lD2n!?%gD)O(Povo9M<>lz>E_cJqOFSCfgEkcan8mtTDb#UcL3SlN!;Be{47Q{ESiwqsB>`17EMS> zajj)fv%Ao=xeLKd7i|)a(!89=JkDQ}d)GBQd9?BKJQF8up1TRUkU!ytxl6v6f7cz% z*%bI{<*npVxlUj?F(!m!3C!IBSI4j)$!`iR8BL5O+S9Z0Fo@(C4&ZT2Vc^<9>PT)!# zz2vEl!MdEV@AI#@8$Zh)9|xuXE^(F|QzJZd_?`j}t^dyG|0@3FM|WxL45I_T2eR>K zquwOim5lD@<83=k&M^+M?I7)W+YU3-Kwov1aW#Cb@KdmwNV!hZxv=f9{P=>us-L?6 zyQQ;XfAQS9GI(u04z7x)SYVjlCV8~3ORpmP1dUlOe1&0}ZrK9s9Am21Id}~0HQcpH z7p|j%vv?k*%ZCQZHug$%35!VoJ}^7Gayz~{@bQAL;brpsN`2K@_h?@-a7hAJ^}b_h zrTems6!%tml42t@RCmRi4V6PHyOp+pxicjr*vGGjZ1x`p$lz$+Q}sf?=2QZ_Ez>hg--RR9O4X>`PKQSG{vvv;k~ zhC}!mNWH=+Or2yFCS7KFPsVY}e>u371pndm9S4RH^c4fuHspLeywMPNqoLlj$PT)T zQe!CIC;$$i)9b(wHU4o7PagH^41mVs1m^$@{5{E@N_0j1jAKIQRnUwpx@mX)i0J;I z=90#teMD!G;zIl{);)9&=XpvfN4`l8P!{QCT0AP_4RWpXBkKjIQ{xX!pRYXt)$1iK zI>n9?{~v4b9v@|OE&T6iCLuGEge2qw1T~W&kP9F;bFpA12`VI51=4z|eN7;}Bm}Fm zYK@9a03{GqMo?>Ln*jBg8BE1fP-y!*1ZXXaS`cmPY0n`ET5~}z3Ye&v_q+BpPcDes zp5Ob&d}ii(_GRt0*KM!8_S)h2P(sm1>`Q!%Z&G5iOqz-f+>6a6w$^KumG5`>e)3l3 z-vE8NfrU6P|2OEPR_Kngt$`-{7-tvbt>bx;U(IG z=dXN{`nA~2Nt}Og1P2}XWu&dFiqz}_tUHPK>D^+BC5}hzzWBhE*mf7LCz%KBS7~Dm zn&*kT5kFGj5)*b=|5{@g`oEj!0{1_dSFtH&eQnv64THuhxO@+}u~Ft4JK)0m`Dz&_ z{_x%CKi2Xl&t6IUt<$Y}j@9%00Bx3%1FL~~R?Mr+hc(Z2%q4e%pHAgDWr`)Aob)TY z_sk9X&{p1n4z}hWIjrY@)xi9h(T9vd*2^L<%9#_V@PW`iIpQP95&tXB7puL-Vfgv+ zU%@+%kt4o@oOmb65r2{#@oyxXIpVQ^miNmEsgaqc!a(tS1;|!RRj6x*U?8a{aLx_Tj{5+%66D@ z0y9L8rqPGwp%;1j0b>$ARQS%5#wowZG!@Dk0w0|9+sR%A`8M@`;mZqoQszhS^AWni z5s{~TL7Sm7;SZ6G$H4iGhT8Zdv?%gPzDwA5NGiAWCVkIU-NKz}t?c_)by`=N^%CHL z1HuE0=K!?YO5PLMGby!sXO-V9>!pFblNOzdY!{lxX4&CQQ9fBSNxsc0`V-oi1I-Oy z*NHsA=IosdPID)zsx9zI26rA8ecx4G6zQy6bepX=ocl^L(zb0XCGS&dsI9p)k-V&< zy|YTm`&3GN>mplT^CH7&7TM@xPr?cA2Cwtr#Z#&&*$4j|%5R+FGuWqF4xhMLcZnP; z55ikI2;OTnym5g`E&{LUSFwK@SO+aaHVaKU1%B+KCjO@bzYF+Xz@G>Fa^FgNazMlH z3BvCQ!jHYI;m-qpXwD~f3VzNVG|Wv!sjGVMI5hlfs=h~7k0~|9C-PoDFXcJovc`Jd z*NiVRIKFo^|J%8LI1GQT1DA4d6k@0oi?AqiBHS=1VoZgMZ7yS+gU|Xpi!Hp&@|pdzJ}k=Y+Lz?$OMLQcQ^I)H!n?h^BmRmzpW3+Z$xAN2HQ#jj8wzF2h4ed@G0(v_Z_{HH zTG#sWS<^?e<9+dq#0AO()&e`RA7S9e0gjy18=CLT^5L335l3JoG|>x9d@6JRT_loE zYEjgvx)S^7?h0gmrXn{_Fi%?}{3`U&7|heQ8GaRdU_laB8kJZ#(?06pijS5Zc>*}@ zxj(z-%jyYD7$uHP9zN1W?J=Q^OAbm~4^uZaOvN}U(;AlF>@0C^a>Hvv8*X^5m7IER zcunXh8D8svK5CJXKS@#kdy#E}bC&g#%moi~*w=tg3QRibvxkA71fGusKcm3UXz)v1 z%>85FlVO~%kbAlvf!MGv}HzA{ttL~7Ydf8S>89TRTwpJhJf zuC8R}I)%Az3%4~VGuLh4E15Y?VUF1^=NWU%I6uDSM*pXk^Zh-v<*LZqZ z9E%F81J7ppn^unM5f~cB+77ovpJH1a!~S^}Tj6%}fCX16wvh#c7Z_@QVMP#znjj1{ zz_20+L(L!si#9xJl?{*+|2xZhk5!p=IbWr@!RlwnC5{}bb#_q@%^cMfS0Asm-sKSemBD7n!lMl z&ELSJSR-2ey$3jjhn_@E^z9EpZVsoPDEf=0-x%m&1biL>J(zZ~*2$KBK);F2-ce@4 zFSpAL8AY z*%mhk=VYqPiQUe+<%@dElDi_e&LOvAsqcP4$_OBn_IB~1tS;J67p=oQYNV4%s;Xk*{|2A|{1YL+-ozD8PspDMqK>SA-L%!Yb*^9J%r4|i40SZW z4=OW~G7k7eX!C&Z`=EEx@-9go?s~S+A9(gEblI35Q%*^K0TW*0ZJ%l#+cK+dZ2@_P z?ahmkQ60o-l*1P?k2)uDn7l*(>m0?Z`TYI!!Pkx8?FR7oO?1Nb{yyZi_SMPypmwZj z>$OhvZ6+W1PkuO{A8OH4O&6?ZW{D31JrKuyTYjb=yvIHfJR-W{yZkPtj^%?nqHQB|_q;QLS-<+e2x(e{0j?X+P z)Yss~}{&!dR zX4YoLvbaq;?#8g|eJlshL|?M@-|4`v5So;JUzu|J`y^&vo@wY zaTI}Z;DmL6j=^Mn#jRUCe(|?-Fz&msE1|vrIDh|a=#IUH0?G5|2*|j5N`40&8a!ve zF}VLCyeoclfxFXg18(P9=du>yCXNptGi9g>BrH=w_-#5CtrlMVGjsAWJpS^#B9|rB zDlU*k-nakczZ?IJ@Y>xgl=cnckJJ0iV(ZBBr~xlQ!RfKR5UaTSsEa8_+4@8*3%sRzKY{Pf{!MJ` zKaBB*VL!`!8-Xh$uI+h|9H9;eadpVoE%Z|!ZKNzyCVgxE5SqgMBx! z#*sC7EwLB1(42R%^8J_`Mx2q_SxZ~e?q=o+oNiBUoX$MP1@Q4c#hfpwWv*rYaDaSz zrRX5h1>$QET~LY+5?!ze{UN&G2^X@4vB zkv3$kskX*DAEa&Z!4=`x2`A=1mo=-wo@NpIiC?;xJ4;p7vVS>``|%zLHRd<7-`mW7 z@1wHs`JSqJuIACihCK4Uv4^=XCOmyzNOsoog(|0Sp>c0t3Hj<4nttStFy(LBkN=$i z30J6%P4@q@tDJp<#;wczty>MhK@P%N?%hSd1k&SMvghEtebw2MiT^hzBc6K{YwT5H z;df%aN3g$>lg6`*(c$Syd+k*P>{U9WSr1fp3iX(pDy_p8Vq}t*forQf&NcB z@;?uph>x^^`tAH*#D53>oeu5*j7gl{LHq*xHm;$~UX_Ahw}$t!ha_V=!5Ch{rnAO4 zsb~3-X5yOJ>#R$2lG~WEy4d3`1ix~A$(gGB>1`$1-vobefWJ4u-woie1pK{P^H?JJ zXR5$O@nrJj{fROsDRYuChbVKDGAC-*CmsdI`OxL7F|PEr@hZDg`lGKsM)@4iI%X*rQslydYol48I`;-*gMR(2Ap*~Yt!>yA-~Dj*SS2z56>P-ZVB_XZe{&JU+vH_G)oMrBk<-#^eX?w zj!lCu3xTh&ZL%-Fx}s+Z^$OdJ@H}6l%7M(c` zddpSrDr9(Z>I1_TIzkhtHSx~Dy<@3yYI@U=q-=p-;#PA#o~jkPT*N|$b4-6ZCuP$n z4x8?$tgdq`DSI(>FQ^{wzdznif=n^yjxp4z^?{(DTC?6>yV zfUN^u*0Am_<-C*7>X{qxX9nr@tKv|RnK|W3v7GU~nL|4wYe$yn`A2;(-=Ayep4|=q z_Q_b*@ptc2zPs8zIZd{>x&-(v1z%(vc_~YRu@$-GIcb3h9ki1sF%U*Vr5pTWr};|6 z|Igee(7xDRznr8b$A6~Ksl)E58%3FWM{q|P>%ao)B+;*f?}h}{VCc41+nD}+?i-QS zE$nGZf9=G!$k?l?tM$<^^41WST9NMY6T{Qp#U3tpZF;ATyGRXnrhQn`S-^0_>h-`0~?* z9tZeL=16QR&Hv@&ecYLPs_c&8z8}ETFSq9S-WzM`*Ddt3iu%{252f{s+3?7w-MOK} z84)iN5l8HAD!II5PoMvhm!o^9czP#e-`9~payietfFXr7dekMJ-k6E_b8GYbV-45A z*~A2eqN~@Vj~_Nd55CX+SaL3-f%VO1&RPY)U*YAi)~9nPbe{_C?(#lV&wl0$+tk|n zS5k0Wql;iKC-W_@3M!opFwOLuvsx8`PzPn z(BoXe?-@&$WXrra*upCt=_`bGyIDhZ*+X>Otd)zyFHybkqhE`H_*qG;p5SOLvR3?N zqQACKZd*lX_VQ3;>LdKV1~?=x!^>}c;4jD-6ayGbC_A^Nrrt?E5f3T`B}a7tP)yKCHMJ;E;A?t&z_Dcp7~=%T-Prw3%Vp>XxEM*I>&$r+Ttv z9&KyzwLs(Jsk>BcD(sb9Y?5R2JrUdHM@~=98$5e`bYyxiw#iuLs3Ov|chY#a2D_$< zyxQWUkr<8^`k92i(#gJdI?r8l)?mAFv1x3=DtEm^*@GRx-81aD?+$*tBiH#u0D z>znzXtkgk?6UzrzJ(E@3)Jf2m-G;Brh8(ja&q7$^4P7sC@5-Au!JF}=9{=6+e zAouxo*|IZ#WlOAk4cxB-&m}?JUk~oz2;%;=(Glq-;J%DGk+N@qf0+|`{v7ykz^;<{ zp2eBvdEj1T_`0c)>FdNMW^Xv3dU6kZ{sWj;ztWq*5vTfb#LHfIhr?V+dFxaVn^J`cQ$eZQn;owo1O(Dfa%4rOfUqV37s$7@^k z$yCaGGO(Dc3F{4D2r zK1IKO!dZo0=Aqn994zN`J;-S}4?};&-vM`fS)=v$|M&5}6lm-o>3^!q`UT^NV7|-k z__;Y}F7bVyN$lY>#=-bJZtKYuJl+<>qu^^Ec+3Egl{~u*8D;Sf=R|!2xQt9+3oZqJ z8Q|}FjlW2ZzYOpfK*p2+a|>flvpWw?M6b!bl>q-{aQ7Yj0bQ& z2Dlk#+IfaC8o3vz*VDC?i?Rj3oGU8M!8agn?qdyh8TACeayO&gFPj8zJ>ylD*pwpg zjd7~@)5}d7u*xI|{>P~-8QWs?t{xk7y)6xYC2dH2v^B0t>F5H+mP zzsqi`OT{1Th6ZvSW=z`NkZxE@>-A z|Nq^1UrD4<)k3aifin*~ZLJ-@H~11BTaMk=g1(ot6Td=!y~a3yg#412HIZMp!FRu6 z9vf;*d*Ko0`*!&5B+pKa9-c0~4B^M6$S=`t>v(4AD+PU22IUvN1K*>}^>OZ875P;O z|GkF(dZbg$U-xrk{+>x{>WZhYyQ-FPq$4AQ9$b9qf^RSIWq|Kw`bkAM$Q=$z=p!v_ z&{GPTBWsxy>P+Ol)>Y7TMHG9iX>vX+JfP#mDYJChB?}{L>Pjj1RlcQm=3eA`9sVWO zWBUFlWR>h?#9^l{5Ev5qt#SO<@je!Vam3i2ZigrJ`!8v~8Q;|&d@sj|9ZY3B(#N)7 zAB)h5;)~O5zBQhhSFC(p@$xcemf)bmnUrp#R&_>uMBU_W|K$!u({0@jZB zGVP{c$n+V@dT0#3e%eBA$@#ani_}Ir(=O*1S@*72rL%n1rD~(tC}LxZ-ly*RRV}l$ zP7%9#8Fh+y-@RPr;A7gZBF6d3C{tFNBfjbNtGrkH+LsecM7-72l$Cjs^{Rg7z8V-L zMrxFXVJv;<=dD-!ME8qqmHU|WGf%F5_O|(d`>N6b_@x|9AhWKTrEyH1?W;UCU%2^v z`>GcFhE|!zT>SXvT@d@M_LRLJAEGahftPE6SGUt>>wlNeyI!k~+wftx_rH^RIaVF7 z+?iqGS7JhSJFW8UlE4f6|0-WcUxDMW_^8Y~AfvbgayXwzJ`sGv`MCHv`N*C1++S}W z#wV0d2p{gxw6jl2Y!SYg<~U;I&!AIBzUA@%3cFMG>|c^|&&>G&+eq%H7MYX~5Lxs* zvgkOn==KX_(eoF`qC1d9(FtlqRASUcWKmSVENUcw@L*Z=n3hFR{j%sdvWRueX)p3I z1)iK&owkd;GUTJ9yAeM98+hVO0sg|+066h|nNK8lyxiUFB{d$g8u)$Yo_X*{HZl+& zvv1{13XqIb5?UcrjqaJ>*{~YIy9HPC^x==Z?Gm8 zdHX4H_c!4x^S;%_{O=|q&nvkH5gB*ie(qA_f5`;wb!?0KZjt|<$8TBwN9_4A-3HHY zSsu2jL5>+hxH_QPitaTPt6Kna|1Mh5l|AEn)US4nJeeAPe((Oi$=h1JH z3fb0M2s&)`VLuYud@s^Qc(%9oQ3Sw+{;xfSbpm z8SLQg$sOFE1Kp&srhExME%GeuC2VGSX2wd447|W|=qyW@k4QzIi2S4u-vQpW>N5&b z-aNbVjI_gDq-Hr{_P>HAWzR^r8=Y#xL7O)Ko7^?^ZN9Pbec1lHn+g(lHU3!QwqgU9 zVk62u-0t&zS8g0Z%tkCZ=+SNGD6iMxo=~3w9oLi$#2ko^OzVv}>BkWmOFvJ3()U!Z zN1etuQY<_n`3mGr*=FPbYw3XaQ=FVPb5mB|hc;*pC3>5~PjTUc)=*+=$Qmm0f;Chb z&t2Fc4tQ=5f1f z4D}tkl|8jp#(c36<-SqT>yAs89huAAr`25HOP2d7DVtUk;dAp_=0k8H-`4(`>_Hi| z9?VbT=ke{#^VZa3|2)niTzqrk-7VNsk>!=ytet(VWqNd`B;#~=yjaBPCx5M{K=G*m4?n(pBTx>_d)6$BH><0SXOFgmK#HL|SOU}sj%Kn0^@uch| z;wRqVT(bOLLD}1>*GE0^f$ss&itjsk7PU{!?_*2_v>o-ZF~6_EyxS@2R%5={M^VGf zXDj6WJ!-zy-}+^5AhV*o1&;OP7mOGdaj?O-V%s|7in=mr7kF~SwsaXPM{sY|YrwbX zhR+4|H3B{;blXSeA~$E|dVPXc%v8H1D)7$t5~>=f$c zh&^P%z`XXuVD+7fKOhNwwVUF}a7ss^AB0 zr|Yp$UhaDy?E9l{je|P&A}Z5~9bs?0B2n#@xv=;I8zJD_?(sJ;2SR)7BllLaPq3V| z&1U)&KhZq+L+nJ6+hXHM*)8Z04`n0Emu73<67*%`*$T?%Q-05->Q3RY&6j!}FTv+K z&%37ndgOHj_a3`>KD>N!_8air(Q)WW`bGcfdRIK~Iy4v6ExxfU$cKCqKUbeId0U$? zxla7Nr+{7ijA?Tp@I}y$@X8{7TW!Fb+qUq(fOsT1n_B=p2!djZH!U8tX+CBwymP_F zCHNhd@|$?ExEk~+`h17fU!mV0)9+tLhYBy(v3CT0zZp&(kgQ#Dm1|om@ZDy(>YCY` z65qPWjy(8aEbkVJEy*}V9tfQ+t?1HzwR!v&o_2DMNc(R&KlE3|Xs2zF)vwys&Xk~^ zyB#=|(0*ZP&=<*nk!Ruy=oVcLouk*p|Gj>dNA9mT{oi%SQ|9f>WX|@>*-~;B*KPi| zZ%0HV=Wjv{|L^Ee*YP^Aoms_rMutt6_E%mQOCz~yiA!PpvL7kDB04<7 z`o}yQ zB{4W5*qi8*1+wOlvh9?eBl#irPV>QU>(i1md|hMBv)gX+&^d@jNyF|;=J#2#A+Qxi zr-`3l+Ogmn+=s}@e|D^=Bp$|Yo^AZPvA!keg#LxGzB2BAKi1#;KNxE&^JdMd#Ffi9 zJ9sAJlsUg}PGvsFh5l<}T>t-Lj5Ytt80}xZ2NDaP5XY|RSjN%Dnq$@F_z=)%LNB%; zy?ovJ{;7kHEa)jvzNd>GT~Ob>YQd(ux&;kuxl0o{(vUy9%7cyGFb#hQ|Ie^T-7p?o z27gaOI=Kq42Sf*wuctbd{sfLZ;8+YCGw^$~a3_iEX*6Ww7nSyY!M%CZX}A+P#($~% zDRsq@OLlIWBJ|id3eAtx_-^=^ z_~=W>eQB$jL%&Al-Ptm(NsNj8QqEO`Wi~M;8K0c}?*MlK&y*fB&J6nN?U?SVCZ|tz zN2Mp{%%h zau4Nd(>=QrICq=^u65a#X+8tFBIQ~sSIVBF)D;@}s&l!zZnP=Wr0#{b!0W(!bO|xy z;OxM|Mob6$6|JnXWPNg$exifzH(*CJU{A=oWUI}=u>Y2_t)GTvonud4$I(#VON@xb z-H2T;aXB35Wgl>8?4iVUIFRRa*yFPJzRlrXaNl{;wiEkjFF3&Wl_Rou#YJQHDK3)piRi?p!?{OU3E%X%}$SK|MAN{ELc)^{~{vHClUt#vnRDP4z{ zz#3_j_EnHi(`Z-vnZUQu$a{v`csajikCV;p^&6ke@a?fBbc-!;4*NmF5R6^Squx^5 zg3q$bnR9H+nDTVBJB#Pime9shblabWZsA2a2QPW!q>X*Fpz#jbZelN(8}Rd`bF=3nb2BA4H*vr$b8|OyBl{u$?A*A6 zbCWOp#N0fIe{1mEjG}!Hw%82X7ygibOh4%t=UK)t__`Qw_Jo+ZK)=Nt?qm*$v*@PX z;*W>SVSGSv|LIt8|Jl$vl>V(b{1jdGF>~0${77z#i_KpbZIrSfFLfG2tocjO^LLm! z7v|!y_(ph4?!A!yz74E@2G+p8d;HcK(7}2yDc-1R1@@1CeT(pbroRLY`)l+I{^&P8 zz~V{!L4FLZO~f4?AnsalDD8vCHwBL(8!TH+@?i-M z-&k@b<17$jM|6|K>V0;52)_SNWY{p|Y#6b21F?IWR>IA^j*^2thqiO}PxD>H_swH7d}Gip zSMq!3!x_Gr*sjy*CyW0dLrZH1&5cD%ZfuHdU|-QcHzPGIb2ItaVYQK(%3w+CL ztM>rU#GY;qGn#8?3m?G_$$xM;J~1zM#aZn$cc;|SiCRuZOZ(7(&~h^4JOQtHp#kw@ zNqLhu@C)-URLU~%(*G#-<(oe4+v%X(IqZEscFqb2>;kvIFMKMzA~a&fJD$7gI_BhB z=H?pqWoGxsKT0{F(OPKY4D|%(a&N9RT-vIBx2d745P4sD= z`qTJ}@{8OpIwr_3LhqU$dop~BXn!eVZVWf+5j%XN;}Z7%o!IE`*c#^M75Jzwz@2XR zC>P^TNvl|rox2w~#qYV$&l>3G0BcN{?|h+u_IG5y#g8{Y1LVHs*?ge^i$4ePQ;nQ` zk+s$Z{Dh6K`RT@9PgJ@Io-~vfHg1o6N?};KIbEozJ-2dUjH{dDsuKh*(&<1ElK5wjIAXO zq$22#6Te#S+hKa|elg?jpLaL*IO98!3~jpxY*fKrIpbJLT!W?4Z0rY#pCcZfXz8;( z(~PRu;h#OX8ddA?N$tr*S5j`zRm4c}U&eI^`6s;icC50fMTty|D3TDPEM zz};!LQCpu1JWlYP%OqD0Ta3rqo<_EE4?nnl_@rxQNGjs--7P<8xnP5TC>sSor+bpneXF zR+(=D+u-rL2aSI(w9`L+i&h5NA%c$!*IBb@zvWDrrWL`lth0U>9KX=R41w<>>|*#2 zJ!hV)3*z1w)VGJd1FOuLB;?S36Q{RG8FHYPle2K==cPwlSLZc1hF#sfCx7Vs8P=`R zKQgNy#+kvsYejz1Z|iE#57I^rdZM*}^F*}KdOR$vlxN%n#Lv)mfWm3H-A`8{CLbQ2jIM{u3m?p7PWFYq4o zi|`gTg0cJQvt^sjf#a~NWm$XSfn+ym=LUGA5rUV53f zTHg!H5?<$d@t$wO>p{9)%-Tzrz0ssg+4GRTwphHlJ;T=qeeR@#P> zCLgS@U}G=#YVc&U{=YWhZ+R@(p6De9wx7VY7r2Brp5Sg$_y3q}ah7AJ(8U4Xi5w|M zwmZlNU5;!QJtEHr+j-X6mW%m-%yo~a_eF42{%4PW4}0cYpy9Wc+@0OTdP~*@qDNnb zu39J~_((}sJ1pGpb6+@*1806^U*~4#F$iCbz?bwDy5|PQkByxrz9ZpVOZS{$Obz*_ zUTH`$s+N);v|$+X2wC5N>@Ot-uVDo1E4~H3e);4z=^#huRNL0EftAQVFTVwUVjK4B zH1vgDN60!2nQQ4a=2y$nz2b+ndva0}@eKxj3ya}tkw0IzPFfwL!wYpnEO3cV5TDgf z%2?wTo$vzxYp-R$4x4Kw^U(mEE7qP33)n*$O!-(qBueos81?Ag~|ZNjhuoDb>qt%3UJnvIRo znNzQ`fWl5Z#_!4Ruqm<_! zDy$9weLTD^VC;dEhhsi_blxgWQb^HnOPw?B4LGj@KLK!5_M2=m| z?*NW^XMEnbW6<0lQo(h{H|fXwzS)oH>IJ~A_%CW37M#MO(Ns*M5(MNxKHIr zluP&*#{_JfBILZ}5i)pw3|2zGSm3r~kL-C2for|Vrw`JW_;ZEE zE(S**_xgGyUzu|Gwtnm4d)X4-;>0Iv!67je>A+FqWGyE=ae@4+l6%uxr#ImfkUh~R zXr}aneQtpV9_Ef!axJp&t?(yEOtl{0xS{PzS%a|_qQ6?0+acp*T_*B=HsfrSm}kna z1^>+XokII6w3f7irks>Ea%f`?-@LC(;6ulR+v#d!Kf zXV37^ro_~WpHGj6|8izc_RBBKrK;EBoJrMjLzfN0>m~S-!E1O><}AgIBaS6zDRxrB z7wx1&D!4`u1%}2kU$m1t8MmyB)Qq2@)GOAKc8^Tf-%;f$(lF_Qi) z@@=1{ugdJ)-Dd2V#LJ0%(>SsCftW-~hrWL)Jf3LcT;dm5dvC<{(t03uwAyi@9{2{RdE|}6f$SeLo?8P<~GWZUqe6p#yRLH7JEC1vzg?s>zH8pWli&s zvy{bWP~+yz1-48ZvAP0dK6m_So^3Q~-(Vdr&sT7FlYY(~qXqjeU~j<()JT6a_IzvX zPoUG$57yWPwjav4dc!hTQFca<|JyBm6i@7Lb1Q9nV@&$&e?Q2c*d*_)6e5X6ek>gMfmy_6H@}?K2 zac-cC7~?`>j5BPFqVFmfO|41P`99=kXMEL|e*nC#&I`so z$o{-l-lGSk&7Z-S@Wv4PbIXa~+zhfm=L-(d4KjCIJ_x=?K9+wCECtjPf4a=E^e1vb z+e$(E{s214vIk_o#lE-ffTZh2djA`8VPzJW76rO^FE=6D6~^3l~pX>njrS>gt&FCkY& znYlN#0=fl<7p_@{*ja-Ar7~`8B?V5Ilim&BT;gd2mbWbaxXZGY;sT|DQ*95)SsF91 z-Yq;sS7y3tLt_7G3q3g1*(wx4oEtU2PaPb7!`pONWL*%Io`U_*b*oh)J> z2yb6W-Xgi9RqW5rS5Ms6rlM5~ekO5W z#Cx&k&~aLeki$}^4Y*EFr<6PL1mBgkDRl+slINg{wj@4`^RdMsN*d#vzb<{A8h7mWj> zz7+@dDZl&1+I1Y*IsC%s#u5kiW8%PAgMA!~0~k}u;j|0=s2)# z;6h?CT0bOSiLtYWgLdNr+fShP$6^cNS0vYSb;pm1|H{bAIder`j;x6_-*euLc-8G? zt=YcWa^|cwld~v>KJT_dc*cJ5{Iy~$3cnDm#u*jz*gr7QC;qPe;PzXrGqLG+*M_Og z5Aa2It)A`g-g=EcLgIyq-}SJ!a||1;R7KP!5i7g`+wRnF^ZaraFvf^J*lv$HIJcrS z`;~F8)UP4#We@izG}b&;j~}^i4ROie=kBgyQWQ{FzCf^n0@Dg3zN*lvC#ksZh~%AJ}68Zt_bA?8k^D-pUhalm&4!n>M%uF_09(d@;Txe7pNN%U{792t4ibo3Sa&Vsps^ITWbW^!*|afQuRSI*fsIj`RcFAhGd|Nkuaf!O7o<&ibq zm*zg;>~P_w#{80GHFfdRg;!zk7e`RuF1n3*vC&5Pl4E*aUMH7=MB)?jCoqv`a1{D2pWimcgl#U*6XgCP@-?}P1o9UokUPPl?;DYP6ha@* z_4Q?n&+s1fP$%y)c)pagN|l`FcXGCBQt(XWR`5{A*(xVzt0dRmB=RcA`R-#p=bXPU zqW_F{WcpIh^vn4nCu_w=^jXTtfXtPQV25h$h=6|8p@nQmmH8h z72vFS9D0HAIe|rB((fjl@2($jzDwhsBNU(Z_sKQLJW2am;F0GG`sC~+{lB`TBU^kN zYr&rdgPco{a}+(&ZVCC*vCH~6Z{5con|&F?ry?`N-*72@J9P)4H zbRF^BJ2%gn-nGHAYi^A*(2gBqBs&8o@O28Z%fa~&4Hsv4#P8TKp&+f5+C7v4HqKi`JoCq4hcB znG{-YWv}HzTnNs#k(2htTMhrO7ufv%bjja2^j-`rhq&ZoFEpG7(<9Z+CU7YKi#TWV z$X=y$#>_d{S1fYY)YdSAu&L=|S zB6rMO2GQxqgZV>52Cdh09vu*UFZov@rbVSU=S{8?n)Y_~6(51#Z-d7K7RCP)!15&taqu~2f$8|v;pxrj zD#-^Sc7*UM0wwQBdX;?!TF8$?99`Q%MCu0R+us)W`r8i#<-H> z)Cl(|m7cWE%oTBpamktO=tPx1#Wj3-5_X~Jk(1!v34Tt3Yc(Qq`YCJ@i4!`>xT2@T zq>IgZ5`K)aMY^VP22gQM(0EwQ&s~B~FO_=$`bLVrGDZeF?^_e5^n;X?e{a@#!at2=Hf4Pe6&t;|Gd9!@m zd2ZSaa_--ZRY{?5*;CNZ-QoSuF({Q+&;2kr-TaGtbd!hkGo8zf7`m-8zAK-oW%ePmga(9UirIn~G|*4@P! zM#)#`xLoz?E~^sFy~) zrU^sqq*K3eGxqfao4;#92)Rr-FFC>P2afg0rC0f64b;9;`Cb@--OXKd(B2IC%uJ?U za_J;JM=OYh8keBg)LFtG*4kQrQ#Px{RiI(HgfiB75G&t?rseIEWSy|StTebz@P_L? z+sI9SFSIE64RwF($bm{|bOxu%HG%zdQ)*D!R z(uTAxIlX0$tTUa~vma7M@_g!gr7^xcM#w#_W=w`+zEhxuU%dYLNU7To<7YE`TX^0U zFKd7x+|gJ17Nb-C&a;dA;mM&%yH50oi*mWi>hzVIO@2c4yeMOovnW>si_i%=O5f4f zDfbBA6G){mxldchbgay}uP#0?g7O`~eC-OE-hplZ1Uk{;0m~n#VPEa>OCRH?Bk*19 zF2otw>evm%?;*D&CK_3>J>MM*E=TEDwgl=*Zfw~%le4+EzW(`B*}x`iMd6u+{4Zs1 zRnG9YaV|sdPMis9WwH??gZ+0wMlsyL;OJ)tRhdhoo$U%|8;DI~I&3X@w z(sP%2*Uid*_b|in;Jx?*th}^te2t|KGxzG$o8^P}-pUs$eoz@V{+sQ}8|w=Lc3E#) z`Q&9@Q-N3H&tIUwzWtI*)4VTRDeiZ&lMfqymU~?U@7VCrbw2(QrQ5yO9LwA#P$!mo zUe5Uzg>Cf&`NH=s$;_7XD(tiB=V{>h9@Y!ukCn3n0?RRM=up8Q_rSU}9ufjA-&7mr zY|G$spT0&8d)_DW{Kzo%!XKC`p&vb0t+Bpl>bS{$evjn;=eh7_Eig!a@N-{EOXTRL zUOz2~4G1mack^Ei{#4d;CQJ$J^L^2u+zPGz9a%2;7!3Psz%I0MKlrX;zcCLOh4%$M zb8hejFt&ZVTx_69=B40^`PrpieJ&pU?1WcpX)8BIotCv%JYyDHS#o{WlCRx?9W2kr z@n3YU*kgb@SEIu5FfaTIT6@pzJ>nZCZBK(xXI+d z;8l3;9{zjaIbvJ&x&S$bOpnp>PyCxRWAG~x6WnU&PC3c{j_oh|Y!~x&y5$TW`PNN2 z9W6EoIzCS4FWQSOJ{%qr+G=Bc(Z+a$Mh4GYad6)3nxAakC3SH>spvbG>2CPq+yv;AxP&Ht$Ju&b49X&G9Knq#i$1#EjIA#Job%bpfC^;5d~kF( zcRN@z;d=hxMQp&a5Pk0lqkDi|cM;O(uP{k}uiJd6E!FW-Nw1 zp{~|u9BiYU+U&Ky7%h-AuTL=Dj?f^b0F>lDP;!oIL^Vs9)SNzxU0m0!$TYTLa`g{jit@6^p ztYPO+R{EBHHx5@mDfccsr*m&H@6yJt;OGoKAlVzXo-c-G{*C9KEX?;yUI&T0&~!mL zlP9r^NDx3^8LT&nZcY0 z&B*<)HPBEi;})L(!$DIP-vy0bEEjPsa`hv0rL5gFpESln3lirM8@NNpi9U))9!!U3 zcF~r=DzaU8q?!MgUJ`rW$z5H)s-X&&TV7^k1NA!iv zZEWCv(P!W09+NMD`NA_ZA7TH9+?0G8Vuz4d_MqhZt_60HzY2LJ?S7wjn{2UlwX};~ zVNYb}xfhY&a%R`$kJ!K;=E)tF34tfL`$Ep~UD&VSC6#tq^DU5E?o0SCymBq~s=KSw zrpmXiDsJkUdK0EteQ$OLI!$ctR3nyki><017!5<^Oa>2v53_&sp6q9W{fj(o0tP>0 zkvoi~UKz4l`Vqa5MxOaJ^2|GFN9TYi&%86#+*cB~7V9z*3)9FkZkJHF6pGJQ9A?3&~FXj5{P*&H8R5`WaQ*yLi44>HT0Vnogk{i81{txFYWC*z` z8N*uc5w8>3mS)Jk_0H}C;DtT*;GH_^puwFwV&e?X1FmKGjWbLc-WrtQD&Dt+vs#N- zOaB~NtY8i#zCqRjGB09VcJ|qHyo1=uZ_mTly5T>uBh4KCvS*kJd^YhF-RS8}dWM{X z_`~L4&pF{oy}p<~-d9-t<6YrrA3SoLF_BMqw|`t<)0%74t~By(kq7Ox>|^=i`J~PG z8}Xkx?@iq$zAnd&s`neR8}w9wPf6@iT`Xr9((|O-uAe;N+&RJdLH0C679Rf-^t4R#f;dL=B|JBO< zny=X+4t}#~&4S@u{CBo;Cudaj+Ba`s(e&Z%ww4c{nl*V-EBA9sUYSQj*VVVmJ)Lq# z=gQT5YG*dq)Xwbtt;_$-L47?&xvp=FOlzzOZEma?*4$`3HnX(mAwCcDS{^B`%Vv)4D_iE4MQ4;e;oANtCe$%G> zcFOP7vzuEN*34?Vsb*Fucg=PF+Sc2;I-l4}o?XXhWkGZ64SW~X%)*{;Zk^8WEP0;S z++oktxt3JTn)=pUo_(WcR#KxnUCtg_qsdPTx@6Anu|DCY+mHp{yvDN&IW{5}|3|Yi z{Hjj;sJ9^-m~Wk18~b>Mw9!&XUTo^63qO!YPf1Q7VC%ezKFKpAGBW9x$fA7sk35H6 zj9>bx;C#|N!&$d(Z^^FP_-^JKpX%WTBeK$s-039uX&U#Y=dMhv?BQ(7I*C1E+|t&T zTvhD0J98-hFwgS>Jw{SwC!yS>Jw1^Jgm~eXV@*s4sCky8Z-cDx+qWhT)cA z8zXr(re>DF(K?pjNj0-HEMfc(m*<>Eisg4y&8#y8_<#5;qRsuZ*?CKTugo>$?oO@o z){Firzd{v@E!kzy@9o}}*DE;v0DPSQ_a~XtQ;h#Kw1v#pc~hIJ$L=bF7TK#9(oQOP zK8GD@V{TTk7jn$7)s?lB>{`Ws%rCG*xVej#mQDyX?aX91q#;bzS?JY%H|=HOl6d5^Xp2A(kZ z^?mz<^fKUC4m>Zbu&a&%&%?m;Oc0*3@hYp<*mp1TV7htX7zKS?vBIUdGwJ1~1US zzls-PGfljN48S6IIckgTc8|qwYEdyw_=MV#6(x%NpdSnT$0Fets8z9xT)4{?LwpL8c!=rY~nbP|kf9{W2XsA?A}D z{c1-j^YRb)<9+zybH>sGjrpb@IXLD$YM7>18M8yn@XIS-#}>{B zj=e+6vdb&EC!#oaWoqS8zFTR_0e^?U2k$}G{|>wd+|O-l0zL=&eO0t+k2VEmx#Ij* z#~?Y*nnTy0E%lKJUsb;ZU4X1FK2V*qD<2urj?CWwKYhCXV(PyO-X5a=Ci<@p_K#f9 z_HgZg_w86tedE9Tic`q@mXF#>P*S6*J(Tjuf0?=y@?biBx167_DB zxfcB>u@TrJrhWwH4*Kqdenk(i9ia}-VN698<+~>G(X?blR7!lw7I2vXE~Wi6zOl)* z@6b=2+N;b*2{AVq^1I2Va&Ut0XoU9Vo%j-7;e8+cBKdS3Qf5h3 z_FuTKn7bfiw9$p1rt!H8jd->10X}j7L=d+&Y zasE3nS3h5`?uam-dlP&$JpZNjyfyf|HNm%;=fAL?7e|`qJtKAbpY}hG3br%ax0f+J zZau$uxcS^Y+SkVOb=LFW2is{J>DvsP5A$5Z*D|W#_c7lZi+dcpYGk0$`mdyn?5)KH zc3SnIEgA0%{r?Brd5&jay&f1&PHt>}@f{ShCS!fRhHn>Z0ap?Bkm1_4qKG+Xoob&a zI?+ySmyP>6jKF8`kAdyadi(HT+qxsemX?aGZ`uB-%5iWm_P$fw`zvO;u=kf>?=QjL zk4sC#-d{OWdFQcXMmlsOB0wwv&ByvsdS3N#MWO(BjJ(*}Ny6xEy$z zyu&^xe7dBt?#(5!J3j0R{mX}I9jzaRd!AlU^y6pml=7nU9pi{I8^;<8zH0q7HZk_= zzHg8d4d1Z%wIks9$rVd?yZUi&7ae(hXZWVc6%%)@V;w89S$s9v zRuV52}+EI&#$vtId=gRa}bPchRQU+1HEiJ6z%n1pY2`i?r{(8atPIvTt5Q{`55b37z0x+u^JioU9je z#SWQ{4+tG2Wo0co*ZtM&g|Z}k!N{7?xhghyYi4C#GV6_8&xW+knFn30y$V@-^$uHde+opD7^!C?Bz(9ajB+k_LtTJ_8@*)YrCh&uFz#=IJ=| z_`xlCy(h2<1dkoyD~h>4#T*e|aX1hAQ_m%MwP9-tjfg*HIevf^@MK%EWEX2C--^Dz z8Tbgg*P@e3D7zS2xrlOg*rhi1Sn|>DEA8Rh#u3^gcXYp9D)T8k(N3So*q@n={tSgC z-m{NOuY@j^Ll>)5=vCd&#cXscwz;;EyRnhGf;O_;@gnxowxFH72Ur3@SOjhh7CW$* zHhZ|XvjmpGHoFTN)$o94(`I*>HaqZ$&2GUXHoL$x*k-T9MizK18(CoK5?c^l9c2G^ zIe6JJp4c+*QVU+}=v0B_d)!rXq0J}jhj3s?7u-NICM@AYV6k{cU=f}f42ugJxm#cf z+Q{S%+R+V;l2mxN!&|3qu@%^$;tMMTma)v^yA^saH9o{vu+}hw4_T+0FqmtYYySy+ zbYmOK8U{N_+vx#dkeutnOZ%|T3$cy6*;iUpk(Mp%2OIWh2X>1^V?qxF(87|`&6^6D z_d;wr%eNR1TMn7PUQ)5^K6BkGeThscA^!R13H9}5#Nt5X3qGX}iHqvwOi3y@6}p#k z|5Iy!`e*@9=GuRtA4^ue`agYoZJ9+8)v^XI01vW;5Ib7liEW4NR<#HltIrnSJ%=-IVoTp;A6ZA7;)*YYc`2_f&F>0a*_f!y+xwpf zZA#Cj+NL~zZLodIrfj`L+ms)h&$qLqY}%UGf{UmTe2PoXFFO)QQ#&K4c&e9Tw{rh=?+2_uUQK?j{)Ta{)t`(a9@iM&ZH!gLkHvc) z?+m`5ocyQyKaKlS@V&F!NWsS$?j7;G5mA(8L=~mYQn8WjuSegrcEJ?(AJXz7$J{pU z=>^Rd&)=E0Y|^H*^Gi0QEt?zeUWUzajZw8_0{6<$Cw#zN)X}oOtD8dH!qgZ~by~cq zI`YBrO%V?s+|+!2Id)rAQKb=66!oBMQ|N=QZhHUx@(r#Bhi|e!_{OH$ykAQk!wv9h z6|tdb#vQDG^udsYqF;)V57z$x-tIDntv&h&r{05;Jjo}XYDl~h{U7~sL!w&7J@65n z(b^JKRXWX^xR823&fJjL@VHSmlK*c{-9QQg?lB?0?F8{{>mLkVsNPz?z|9zA+gB|} z`ty{kJ1FO+o|@rJjCKEP!LR?b@y_@|k1vRiZM?I=7GL@H)F%@E%=;2Ls?wo8yR-Aos_okhnmmPTn{PmGL`eSIXo%NvxB*EF}&_+LFBbDJj=~vXwP6@ka9n z_UqB@R$uMN)Ty>`>V@liCA@3r&NUmp#V+(OKJ?;iIbXXCd#I8#GhAQE|=X{66b{0ChQ-?T~0Qg+ximThmJ@L`5=(^}nmLGZM^27DdpwlO- z;dOt&KX3w{T)fTI{R((G0Ztx+7EcfZzsCkF;N+wccknc}QR9lDPYTd8vbXgE#`O>T zu!H{?W!v-|c)IOD+op9p`Ao8J`T+S9U_2$jy8>EHNy-1j4nFZw&L3Cq&iP(Nli;<2|LEOb zS=YJn8yVbv;!IZ6=kNbw42E@9Pw)4#Uwb(;w#uF0Yb(R2G8P+oCGpGjd+Zuw4Uh{# zp9M1UGOaIuVm&wY%c1z%BS&LurJ$xBy$ir~(T8X^H#;UswS^FboZ6$Jd z=`!c0ie;~Cns|O*U1ykg1hHU6lZ?oswc}o|7aMRw^6T}ToN*d&M09zenWAMO3zs8P zQ=qYuajL2tnR)|se)6G)#Asxy51CrBIjU;PwDpPesXsN-mssL6s!mQ_pLoYqU!wTv zN>Z`cpdSsB?2B_}S0#I1&{?GuniIOZiE`n{-fm>?4a-6oMk9Mgzr-MW``(^Xbp`Dc z@U~|%%jJk}IUnzF{wdMb6g5U6Pu%4bvy*JZ z)`QQZ@T8*M)@3hjQlFI_Sts(Ca~vm;HNxMfJWlbw1l@GLZ~G7b(@%f6hWQ$QzTilw zJt|%1_I>ug#6M^UHaWjj$g{!kt~G{np3M<>9~)$5Fc$R84dh1OZ^nZLLe>218MBwZ zMSsp??vImSGl_FA?ZEb$V_4lfWaPudY>2!p2_}rV~OXm5#JU4l5>=0gi-g>@yY(KBD z&>q5T|IYtUe)wDe#d6g*5lh&~dgK*kpZK2F@ZE{7$~A^{uYoo@L--xanMj@=pqze& zk59@vnKx{MA#sS3gZ+~q{-H@^+;Z?H`bfjQ_j2I2_sfVl=26}sKrhL&<vIAFa7wH1zpiYre6;g}&Q>Yd6oVIQcft z{yUT>Cuy&$nhU=6M5xoR@I7ayij_5DC1ad3(}OEp&flqOZ;0wmXRe*ld@V6gYo@4b zEu+p(@JXyj-4q+~nU1tx7UBwdzm_&=Z-cgNdnfqTG$}vl2M;#T z4(G>u*ZyDj-aS6b>RR~U&rCvQa)Ce+NCGmsh$Ioj077EHOaj^vG?sw%)=D6H%!Ny% zcuGV}LZXHYAcM5D!Il78GXt?`p#_iU5TGIwwF26=^=%K~qD%<4fMB9P-tT&z8AxK# z+SBuUKfm|%kNM1V+0VYLz4qE`uPfh`RhnIe_^!Et!*tE6@1qkmi$nAYqVv^#BWN4@ zDgC=Dz7zRaJm=zazPCWfE%YamXH_qmU2B%Bl8OX{JtDeC=ERr#NMCzb*3uAtExkO! zyY|kIS6cS7CXu_t1N)xO8frYZ-{4GV?fsnR$dBFZPpL_q`?6+S=gcT6Xv4ASgDmYk z&UCH??tcp447^MHQ+l~BVY?2@b-!zi*6^`~J*;lmbz42qnna~ONbDP@`*_s7p!Vta zGe&Wr<}%jGP-g+-5Z{)%3a@<^HtfyB8i*lR-eI$Hj_oD$dk3`3u#uOu(pO^{7iDbrY8~T^I;yCnnYKm0^a*%kVs108 zAF0R5xgMVxpAF~)U_#7nJ;q~hoZV#!!3UDEvBd7k;GNRa{)oSc9%qF$94>x2dl>f~ z#$AT5j_6UAGj54{p^tfyz3p3#xl@0yjQQ*8_&dhjeY`TyLVtvwbml32_PpY|qQihV zA|`N^prqkfo*m^Gcs)|^62Ga>?#4G_0X~+y6fy6x2fSJAc*xa-&)Y24Zz=h3nxNOE z#6gIKugUxYK6&Z(;$1ugUO9`2$M7t$JBQy#Dd)%-_VZHCOgpGUe@{Gxr+HsWJ{<*p zDSgZv8%e$;j>C@)xa;|J=5cL8c2!FJgm=*O8h!?0?*5_+)yEs|#_oS~zDbQ_?L1(; zlyM-ziZB0nRO)eaSkmN6(+-?a>dcb#p-u-e_F9OKRh0d`1N!v@u5Y_jzsAS5q~*KZ z-;`;*o6Pm#iTZU`h$zMrR{i>Mm7333>(Kdne#|xF)vp(*)H=q39ND>qan348O2LmUHJ`D%7^{`B z7VsYWo$AR;hqi@F{MvR;n!Ti|pD|vM%ZnUNA2a!xfJX`TuWho2SZ>BXceLavG-A2> z{O{7nVT%B+loLlK5SnAnFHCXytI=8P*W{JceTc-z!#W6B(7JF%A~qHZC%I zxcZl0a{jC?)rK8s-AqeNr)}zZ2O6ZG;qnVoj-!`?_n=Q!y&c(OtV5Jbc$(wzj+5k_ zhR+jwmwHB#*HN|0JDba!+s=F6)0z{Uh>5vEyEer(r8>nHNN5+li^Pse?HG!^@eShJ zsJRZYSHzwPc5+tSR9PnZQ8_10P|ij4iFYgY_nZeu@&0jBqW-=j!+3ubdiN*FFwPHR zBX2U^&>dHzAN9v521q4c~wUFTk6%zs@jk2ZKJ@hORBuA}(%7d-ll-4!8wsKlk@Ea5>G zm+^_Zn?3Y(3v+B{j^P6~`7PWv{!8Zi=<}*xx2r_!7UgE!w;b$XH@lNLUB7EE=#`%Th-7-h&ysCx#;fu-Z zZDTLU2Y;k&6LKUu4<)fKr_*LFyr`UCY@BIkfTv>xH?PB|2%Ij2*Rk`Rlpjv{?UaYN z)XPuzmJd!>`^O$k)FbOThX2lSsC&vc(%n}f;sd*@igv@m8}N9H3%@N7ZCZAu>3oFY zcVKbL`iY@^;ny2j??zi){Up*74E(7Ach(UTN%)Y?z7%$4`(wuh@saZ5Y~@cZsb0kG z3}BBXM~&Hp4*-4vG4@pPW0Cq3iJRIEZ+Z^?JRF&WZihX@p1PR0R*8NoW2J9mZvj0a z#~Uz+An&5YFfB-SLW@60d?MK?c_PZZypO=oPTV`+y^KYChU9q}ar1o7_4S`?tXp{p zevVD(V~j_9QRG=7@UBBABi~qABYL|Flbv#Y5}))7^lRo7n913uzdM)snF&1etWl|D zCd1#7Yp3{IRtExiXjLL~DKY&frw8Ar<{QoU7c1u_;{TS>makv(EQ5MJ=Q(qs_wCSn z?4y9U-d}I*DQq>|DpsWq7@|^z-@ddU6L^0YxZeT%Z|B@e9-0Xz-M6!8eTI)e{2;dK z>e$6!>Z^2_m|x6OzSj?2V&Ke!AI`tQz~3t7C)1SD6`mWDbvi0^unU}*Cp!-_7U_3x z+*kKNfmhwR!6`m?)qNMb2E24w{61Ny(E{hqz@nDeR$c1$(pd2Y({by1d#a+}1;m-k z_4b{`)6MS#dmnJ^@1^A)9p}@5^UgYfy@Iyu@=LP`|6@O3@6S8VI}5;j&LS4OJ+(h; zPhctjPHFT2e>2b zDs>O~0*fvq`7Us}19;sIu7an~2Z3W}9~}#D3j==n_!vG8{3M@?$R#+Vx2DU!Jdr%* z&@|nzioLl`OBtIm1baW@9P~l##biCVkHL2uez-A=-GCiD96qG_x1GNsUUB#GB0~|L zFO2@@nk2`bG5=pX@8F@(e}PQ`@jGSC!-WQeqdI+K&IJcDS<7AKY2H`osfqZT0(V*e zKKhgd96Znr;ft<}R~ctSsaNQM;8Yy#%a{+#{^P}&I_Q*?`Tun8V^nJY>t+3a^SleZ zJJTQVB|IPc(}lmXjJ(2M8Fc09oLoz`*HO_=_x}-D*i_^@7InkmW*cXiA7&O>@3&12 z3a!lvx?hb-)Ev=4C7k=u{5q#p3oe=ge^3Y^khhaGmvswu?g^E$i7~~M{%KTYSxxD_$;d8{{Cxf z#`dkQ$wxLP@9X%sKr363@LE+(!Fc-%7IXsEOLrVdJ85p6iCse*^Av?lr0O=Kyr<*& ze&#JJA9w;enzUa_Ool`3WpVcs$Aeg(b>^J51kO6O=KbSp%|9DgXP(+tXO3!HXa4j2 z>BK3V4WC#WFyHkO-yQ+ipP~IQV)YB&${Er(24rs^6nV!`;2lFzmoYeMGJ&D3FZ?^a zqz^`GZc~q8CoxTj6>>O{pVd7#(zU%JZ2oV|;ce1qp`CTm^3Huf0H<<- zYw6QlW^_aq{jlG-8jrNIe7UDOByYMBJK%WZPaDL?neF*-4)A}f4eVkx@Eb(6wjgG6w^|AO?lgkl}-_v8N z#Jzq{m-SK3yhnJ~hi8H=AKm0zAD&l;@n50zd;pe69KC<0Pk%_#DSjzprj^chj|S*Qom{?#dVh zcclNN;BE}-L!Vhb?qYAglfBtb-g-3|Vc=RgX99l%*Cv?$e}-$(I<77D(&8D=6d$hX z@PQ6@hL7+8@GsvQ$LfGfEpy_-F$IoQu@Bba7byI4S--OJRh%oTl2`*b*xTRbc~u;| zt!7V6+W~#YmQHxYQ_Mm2x`vX;^rMA&OXOeHz*6|A8OSx?ok5&>p2hv0c)G)&shcdf zCkWrVDRD}Ili0n*37pL)QExMCCR6vjv6{05eCQ{9 zRmZas-q-g{aO&lv5}YH@JxhJ>^<7${KeHw{Ln#wV?u_33men|j7lF*Ic*ASUYKFhO zxW=(XxsJb99lbbNl~muNN+M)kuvR<6^1_*pRRXgm7dp1+^wH3XEYGMV9|H6-cf49D z_%8FP<1{#5Aosv6f&ZTpr=S_QE{+i>M4)wP>#-4oYKqPa&lQPJ6HaD%e zyLK#^-X=03{k+Ec+)^=pz6bfX=*4EpK8GBvt{?vT*a}!H9-5y9FOthT5LubVd#ibR zo2#GIk;r@DeJtyz&-axRe?Kd*)Au|6ea~Zg{slC`y8fa0fnoM-LCC8%^b2wvCmy8d zJF28PO_k&#b2}7|JqWz4oY((|OfPbma(=!O+j;h5v3t!&wvmsGuZ;J%^s#_(fm{15 zj60lhk7eA4JB{^Fr?Dz{;#$u7zBbA@8+}e2Tlh&{Z_#1r_p-OI!S`+#{GjaX;p{KI zXBogicvMT@a7PpIQZ%dGmEeP;_bh7S8^yRg#}{gwPW+_`pUz4{Ca7_4H-SH9@W>B* z@`qL$`m=NBRWBkhF?6>kWFt!OXUrSubu!IGD+;h_OfuEx2=8{Ewm5ph_cI-DXj{h| zT!fuEd&wW!n-0d|TfsH`*l*$YFZ5{vXELc{E8o9yc{_PJ%l2gSgycSD-m_ zm5rEV662)5yB1x}T?S0{Pv(D5uZ1g`O!iAhkNNjeqp$C6ddV^({iV>kF)zi;O$2W7 z$T!uX`0Uu1Vn)RBfBQ?JBi8XQgX@=hww`}mEWUZ{lb`gmUwVl;Dyicc%KeJ>QpZs0 z*ga^dl%M-D|CQ9goOhvvn! zjvn$4sTa9rw!N|`JBFXy+WL?}Zg~*=f?kLp8ZugoBapb4)BUw%^fs-wZ#9GOpe z_I|e5aYcd`D|@NFb;$G5r>cH3kFv&8N5B|tLc-rTi>&+NZ*r>4YE>^iM*hB} zx_)ZgHt^k;_vkKoLEH;HUo7X7oR#!xkKWekq71=7+RB7alJ7L?^R*RATSi@7aV&wl z?5xc?AC5sA1jpn%S)($Ze>D%9Y>a12(K+nm4ghcosxl>)xFkQZ2 z$TY8wzjEdeqs=$r>&`>7y31S7u|6d}vd9Ui+lezuUj(*B{5X@|Z-ejZ@6Pwq1(6#Q zpl`FRBVR0>&T>^*GX~x0Dv!O^IakiNB5&xAp8H@g|HopmbswWOja>sBkU3xv*t`Zh zFjM$R);ctKe<6La46~OcKqCrG;SO|cZi#U$G!FZ#7CZQlFOz#I`1mSktGW0+{)l*@ z2Z?K3j9*OXNN3H#a&z&iEhazk@c2 z-d||L!`Gk|C%z#1)7xT29f|ve;9|TJGL2S<4c4 zR_sfzwO)0aWmpD_l*jscw5*+!h$uil(p7PFZzCGoqUR>gz zJwief+a2YO=5W3qt?l&wA=Z)8&rQ(w^ki5WM+1UQ1-<{i*ui@&|3V-$MsAhj(YJbeeebAex0%^FkN`ylYJg?B+(P!P7jfxOR3TG)4L26lmI3y9-6dnq;0}0|yo@44qo&iXNE7d&zZBcv9X6 zwbVkB0~ea67CpZxCu?C4_4J+EWBI=tBUCIQulmQzL_xjqXmuHriCh$mq_6y^l`PTjR6UuEV0uMPW_;8_t zTtyyTFFv3}YwZyJ8b1H4>(<8y3+|%3IAhU9P>0FAmhxgNzT#G5V0WUmLQgyEM1-D6 zTtFo{CC($hv)T(Le0AegD*i0U*ssdMCDv$9cCP2ly{N|@KE-#Qz)qApJ)P=ACU(XC zqKWnC)2E7mgxE_wL(Kfe}nIoFViOzgzLPN;W0yTIis#WhALzd0zjf?i zk2hcCIgL1ol6OYNSJi0_k(;#EW?<`q-N*JE>?iLK{OHO@ex1%66!|*$=(Ne;8-%_} z4z|HUi{u;hmm&|(pe-pc-{)Qo82dT8_O@fhTopZgAJtw6{Y4h)w6eB3>)EAlLvLyD zHMw7qC+VtAe0gV`cp3C?9{N-1a{_(#?XP-2&?!pWLi_YSwsqXn3i&#y3%2VC>OhY{c;N!B}*vS797= z1;+Bdz_<+<=XSGa(BTYC=niM8-+*&}eUBth&Nbnj3!Hs;(FKO+KxLi{_>wzb@3Ud2 zE6-}lXD#xc1j>l5#oyXDsmJJ>cmFfUek$k0 zMC8#)_+r{B%*EngW(`yQg7_w%^W!XRlO#7sbyQ%IZI`*Yf;J!LJZa$@3(q5YovNcI!xHMJB{zQJ{3N_Lyxjlb}t=?~_L$gk{h%OLb zs}`N)yywe>+dizM&S>dOwc!EsnRE#zp3>I-yVv$Z7+F$>J9;NgDih1ogAw}^Sl zFEn7*#u)D8ei-EkfJcIRA|GwTm*{uIL+8*FE@ecYMX+QYrt1WLg`Z!6`E`^jhhIHx z>fb2xnHi@3`q%|F@LgkKykiuYr6cneT<>YWxj2{n7G9etKlXsi+!4+^#@9rhKHl~A z6ygkXufJ~){!8Q#h7Fg%IIXblfa)+cjjHSMA2-d@$-hyXzy) zJ9<6zt(N)r^*K_;MeZuf(GI%8n6jK1$%n{yAm8fWu@B1JHjgs!U=K8_ zob#o;*J0t!OM1xLj?bp}!oA2>&4wIV`mPaA!g?2aAph$gV~?KJIiBwJUAC=h5`&nw z*Y2{V$bOYTn+h4b2|6S+C{4u_$MKyn#;}WWR&2B-{;%tREk)?FtZ#`kEVz*Plo4mx zSI!qR*iBjen@79K?eIk1xe;PSV7%S^T zx4)?{VaJbs;R37c67w!+w$T=Q`#iOEY%{nbu>4T{X{;q?Z4Pl7k}}cN<|nBV!ENqG zTENvr%{sDxb8s{B6BrC_;(QzwtVRW2oOob3=izMpJ;aW+FnL+D+w1RP!*;s~xmFLp z8q393qr2aN6@BYG&YnK|P3(X5{_B1;_TT9@SJ$j%y#&%n z!T%s^^Gh{rk>b3BtU|0XK0Z_1ObmC;8CQh#uMV#uz0NGs|a_=xPj4O=yOPZbINMT}PW_8r>U82T!8>_i?Zbu7KqF;>3w*|)QH z!(YA5zM`IeV_CtRl!MrjC}`8c5Pa`|!9ir}vFKKZ^9{1jmc#gc^v*w-au7Lv2>GHO z;QO0+caV2Tcz8D_WjnIbT;}x;*Qv1WTg=58^>sW$UOMQi$UJ@J7vU$Omq)IeOIv#R zedgkHzU_F6^89B|{ypo!e%s4U#ZA!E+vGmoG-dk^ReXeRZsq>fXUtO$CaYqTw4-Us z8~7hB&)+gn8O|KY7!qh(?2P0+o9VRS?A0=!nbaA=c(ya1jkGtMeOjKcql`XI$~U5S z5SsC7l&zm$-!^mcTHgEWOQIjX`VMn_1wPm{OPRa)S^ZUe?{s2{a~*O!J`!F(fzCbz zy}zcMv@3O8s~&kb`)l4y9?#R?vCKX>$~wE{#=zOl@jH?JZ$7yL;4 zXmJg;=C4)oO$a`N`S|M|N9HcLzEKNuIJAb;Qp&abFZ}zFC&n-r%lsQs<=di<95vI) zl~4wpmsvYK3xO6fPs>6&JtKaZU7q#r^elq2AmdpU(dk(fGE1ISM>V8Q_qHMHOvX5$ z@dfdp3vXbJ#22VSy&iK^-5t#PzC52D66~;O!HqZlQmq(qIrG5jU#JzQwBC;Q7Y43Y z4cU%m{`M88!qt{N!xS-@La^gDw+0gzAG@qpGc;uxem(D@pRI4Uuh@Q_YFz~Eq~4S0 z8APV`Yu3=2Lkm~TjtF(2Ty_Q@RdWv!pTWexnV%oOIFo5#JhH24!S*|!;&+(*p7#YG zyz_fpKl4D$obm@_=6v#;sb1s%kgaU|GrgXiTORP_+)tj<>R;ylTK&DMujlz%p5ym8 z{(YXm#rC z_y3XoB>xZC>azd0f63fOY%gbjPMJf4*Jgjt^G^r=Dtj!?KOgMY>wkGrO!mu_XZf{e z7t>BG_pw~ZRvyldqrKhS@8o(y<&o?R?mwlB+)qSbdYF1L25rv%74`2P^xN!XekF5{ z4ceMr%=4QEZ^~w`$-WbP>Q4SM(0d-@Kc0Ft>bRcolTCikSk_oMva|Hc6WOBwDCZZ8 zUNo7#se{<6WhXkO&57vkSm(cE)1{l3=P~vK0lx;;^`FyAr?mhGW_Gj@XWgO-2<6XnQqGtwc;RauZeY$gzU7% zoa8RzU13y~JBxRj>|4;!)+X6+tnzK5{_UiwN$y16%f4o1Em$sP9jztr*TsHP zHDo&0f!k?i#7N_Np(jEoWW9-fa%zl$+4^SuHkC5;G z_^gyO>iV4P)!ysK=f2E-fHDtICg<5Nvftnv-s#`0?DWkRu6^Y~DVJ3aKT)n)4-pq! zd#*J*?zwZ>>sd#~o;{Xb$vWEo?5El4*{1B^3y;1xmI2FcddDv{BWGp5@O17 zsz$1kHH9kmGx#9+MtGql$_w8q>sh|fh1T0tSj%DZEu1WzopP!$H|2TO{TX z_T_wJZ(>s-<#zJyBrxz0vyr%{Me)E+{trQ?GO6FD1+{nvRM#92sK$>l*zt+T66sGB z@1#%7oadW(Ci5)MrH?Y#($9QgBF`<94b}qT=`HRx-hS3GS6BNv+&#w>)Y#zd=Q?sS zc+59?`x&VBa}m!3Po$p*>0bi;EaWKrJvGv9q*)15{FCr$r-#Q zlbFQ7pz+V)(Cjd1cDMm^f%7eLK3QbYX&=sCcoL|{~mA#OCU*5fKdnH?V5xL%H zdoB9`>Jk1zz87BO2NN&Oc?SL!K7;RL^zUD?y(E0aTzOYvTd(sF4^I4YPOhz1=NF9c zLr2K765F=yf8$$t5WTLS+J2k;1KO5%Keqi&FE7{gZCj@{y*X22Ui&4is+h(07 zk#bMiwix`y#4qN2&$cN$u``{f|8b0U{owW4#|FQWT{-xr?A`F%YH)4#?So&&PU+?B z%0b)@dMP_^&@1q&>#}FSvnqa?UCrUYcObO76kEU;WId&UPoI_bAo=L?FFbvP$4G}( zH=BYSg_@-?j(HOt7FvD%?e;gt?$K{$uMyZcV*e<%j&Wk&IHHpOhbJDL^T@XU+~O|pVdTxpWxjJ zaCL?Frv8$3BX}rlsfO#cimN<>JnM{$iPRk@{BDbZi^sjV_+R05|EJ<&7hd-tgo|lr z_+4-@Ey#z9@C~1L#l_>`qVT(di$bS_-<|j^agjK5J>sI?KDa12c!Jnhop8`8??nD6 zxF&XsKY7UC^}!|QU#}|x_F9aLu-q=d_L9Y->-})lQ_QO{fy~6kGt{SgiAsuD8d>y`80v8iez-a~GL+Wh8vd`jj{bX~a7 z-Y&U8%aOT?-o~Od2l`Q06S_iaADu-zf}rrUPI{%ZPCBHtPCBGWZGhu5d;_Fz@q>b1 zG!7t-hS(8`j%z!#td`iDb^RMsqL8aL5%1AsY8zwmv!`0o^QEC1T+jR_pc^D^ZE6t)r+3u};yZ8{zDDfa?}JbDpwAQApPlT*hF-AkijGR;!iN~cz20&i zjOl8*tGcZ3Q*JGC-zs!BhAuD?I2ihyn6m+6n|8f5t_fMx#n01IKRcJ6x+cI>Jb^K` zJ#OE&#yq-c@n?q3qpnW^7xj6^eva7l1J;-Z75!|{i_tc0^Xan2leO?UPYhqe!)1W z=)q^9YZm+DHSoC^%6JE9{KywMt(3>$@SmV}wk1pSs#W>;7b@@gUK2gmQTdriB(@j}$ zz9lnxO-3}vV7LSACYKXQdmf2Lt#eN*pM0sPE}zeaXd|J zOQ2uzj3L3_uy>fg(5(A-PwuPw4YL}y5mix^r0L+nF51G^KQ$5Bn834qv5QEbSgbfd z$lA^98$HTHoi@$T?MmGutG6TvCHbD`(J!0U&2E60X8DvAy8(-4b{vVvCr{RIy4l&h zi(Jp~-*I`)G5(H@4-#9)5SMS9p5ywRj*c^lYj5Fxyvs8|ct2pB-k1B^biYrD-)CXX zTGozsx~ShX)Zf+I(aR;ic%D(%-Axc0?dP5YYz}iiC+4r7U)re0Og*&QrC!>tP1s za-x%`vnRU%U;Im3nbPe>qF4&qrz}&^ z%l11V`qK}Y_8&CdJav8Kxz0*lJQ*EDo!yRf%OtMY5LkTao< zbMYzGOCQb=@1y(a(1MC&t^AcasG<*|4^K)2yh-Q9{?H%){;CccOCAD07H<-=@5+%w9~M zg{xy5cV%oJED53JK-(!u*8aDeA6i- z<9nX+mT?37S^u&!XA-;*ej!CF*e{zsdAU};YrIyuHrsNitsk^;d0gsuFR99Py{%;v zrX*^9Q{uFtgH3~iv+aJ7bM*Jm>>4ls@r!qjSEJX@i5M=gkjUo<_pCR%XRhtp|nb&CeG2VT7WfR%jC&Cu}4 z;8!rQ%$E|I_!R4E5BF&=lVgN(a|K7&;t$33^Cfe({<`F*t*7A6J(oMSh-`ZTIGf3Q zU>mv3hOVap+_L#`jjlCUt}R@n$5e71%(duN;UR6IT%%L9%XK){=rxp@%(LANW{FH`onKAm+K#J{Rs8TbuQNh)DPcidzkCl)GychTo+TnT+ibA3F;TV<40VVP`_Ny z<9Z?W!-v@Bb6sxewrx*w{S5WX^D?d%Q$O_8R>}2J>X+*(uAS5`*Gstm3H3t@Y|FX+ zIrYo+O0IuF{m>QL&$wPo{c`;x*DoUrW{!Jw5NVp=sEOw7vuU)`s|H=khzq zya;{m!G!?@6$2gmi@ z`JdzZ?fmPwV-wfS$(vX^=g6S-*#UQ~n^Vd4QRGK{y;XfY_b+ju$NfaE-{QIw8Ixb{ zwE9qV*Y;cJ12wsM6YT}FB@k`rwnIw4Vvvi*M1PatnfH}WIxmTH1?*RKaj8I6k~x8 z+joYXK>OaCzBM$iF*^B%_o8n$g(jWR)V9;+-tgR(mLu@~midKSEwPQ!mib9rt=|(H zDP#X(-ab$<5l)?MkQZewel<@H_tFvWd2Ie!Nc1I6SNJsx~pJu$lZnfPh^RK1>OotW-3Wqp7{CVVU9d?ECF z61GrxLa(s*aDof_vdNKf8va4%IDq$J+i(=yhV|AUY%TrVN3{UQ8Wq|iG~|<#;;qS? zZ+UVVbU627MU$J@9D$ zsI8B~YXEm_kHT8k%DD<%$PLiKD)?**b;$P_w3n%Z9CcQUqgP4tR`GGkXJ3~0GH277 zmn?FMwa`bgx0P}3gx=g}F*!WF)MjjQ^zjDMP6*}2Zf*qQ4JFr<^i{@rg!w&e?TuZq z*&VNX5z9QNr2%?z{d{Ab8H{l^zj*36$~Xl+{`_{)hgiy|(a)LCre67vZk@{SWfkVw z9sbhlw;1Og`9rq~j1u|QW(v^9DQjv7<8*-siPRzAA7+lM+&5dTjv7 zgKi3#P_O*J6LKXgKX4`>oqzC!T#3pL+#yGz@>BJJI&M!!Zj*(rrdhZD^Wm!xmsfPc zPv5gIhIPhy1CP)1Oy*GLtUKPH;+=(Y_w?=|o+xhv;#ztzU_QEije+$OB&@c9Egq^nJH@(A~cm-d%Lp zpP131BXuG&*7W&d{>Pl2gvtkw`S`1s+U<|IYG z@>mAEO-xMOVC;uR_)Ux(d>me7a!Ex_WBP{2W_sUcG51NtRg1kRVyeJDjy8UKvSWbU zn}*v<KnIVPkMFS zPw3;$ykhGtJ_CPq``zF!Yd-B;wtvqzx4-SbbNsjbSK9vvj6dR=+s7`b+x&lB{CL*Q zx0(O*$TP+M-Dm6G!~9>n{hr7Ft=j+SYucCi`gPEfI%tWnj}Hoe04+fd_jlt(-@m1f zp2zmB*48h(kF7$nwlw5W&_yLNpowube56xs!bK*N01uJ5&@wP+Ver%{^4#RRECVgb z4#`0~zwU@_zMS!>a6cj@o8EBz(^j+s5xGY}p25arCoCwX|wqMVas z-gvAx`E%ytf4iLixBg*8PG9onNS-_k`+Cq*^{b06%|0-*y!XHf<4jZYp6W9&51;j! zRiOjp@6VfGvClR?gLsN@B{SzQYPHQ@!Z$Y6x2OqT$HG2PIz$y?D?80X4EI^%O;cyn z&XR>81DD|EyOj1E^&ta`#uH1iq-g$0`nI7UVZNN*zXF>O+f*NHhz;Zv*i2#!0-I*; zdxFi4-C!eQ7Pw2Fs<72Q27lApH$u0^2S?MGuVMEret@(2rXuEg4X}#`cGB)$z%Gk= zZUT0vfL$M8_anySWGqdhtK?srePGcueY~&>^}??2z&v2L-V3{# zz-}F|6TXpHj79Rjz;2BfcAWe7Ed_RSXs3Q*?}7EeZW--ue69DuCBSZ0$*lRO=-aym zN%KW6ApI``-vqC)o7uMn*oo~)5qNciat>nLPi72{Xn~C_+&7yOlVXVl`7pTW+xG;w z{`bkQ{uVNB>C+Y1?GjwWAErX&Z}1Yx3M{}YZEc^{JkA=DJK)Z{l(W!OwqkaoWB+@h zubh4FjX#~OU;63Uo^Vd|y@EGI3*Q6hCijSQ=1w?g^5Pt@c6#xyMc{Va9N5Slx9(y5 zLN}`M`;fZRcqX_g@ccSl3=&+V?;C)p;3DvBW&M8PgJ(fV=>g$Q@ZC>srd$H>tfCEn z;E8M#K1JYY?AN|^TBXC1cu%{$zAK$z=f%5rY3Ig!7JrP)Yk3xMECr4efa3}3TnZd# z0mnJO@qOU$IQEQx0>k$W81{gB6FT8uh8OpMpMGuu|8y9hFbC@}tOJI^%PnDi z0#kwC65uzx2YlR6-&ugc24C3T)82Hq z$jgkq>Dqf{PdQ&Y?b)67rfb*H^WJo=eYcYz(N(_hb7a82Jyqmif;XDQ81F^+)DCkm z^veCv2Zx~-4o5#6!Fi)U=h^|BYyF*v&|3ti*tg+FP`rbA4Pvh@D*Cqt-Dlsn!8~B=%~ICeUhPHvAJEU;NUZKbW%ktIN_%RMJ)Ky$ zk!?YPqgqn znKFf61dPQgDIB@WSLzWt>3|RHsnH+OZn*R_sx9bd`8Im1*w?kAo7hlsPg~0_OWOwX z_qVpJQrk9Ee7~(_s%qb0zISWO2D!httz|+G@9*2%vYmVL{aah+%Jchp-i!X+-?q{G zz}6ELY8&}j+P@s@?>dRz`}7FpA9-rq#a&uzOPYOK%iCJ&DVy4M>c?u^=V7MQF9w)W zk2}=1b9tuJ3y+#o&#Y0~+J0jKrj-4iDYf-C;;Uv(^#qtxWgg4$-%U7&9vJ<}8Di#D z_fli_kf#n`)8f`XYFhO#AG4@*feKGy;FWsJ`}gatQU+a}nS#7lZz| zc9qz(#)fZT#R4Fn6LXvi|@PP|1Gh( zMNfW{*vXlqivD%c&!hFeK4Nzrq%Td>B|g%h6Z7Fo(X)H|o#6Gk7Jp2s;~ek5(8~>U z+lfmQO?_Wt^B_1xecP)1$oatBSLQQ5=1hT+jKs z-G{_NF#0O-j?C0MiaISe`-(um5g)vJ$`?mB1K-j2+g)N~5kuXP$Sr1ITewHB!}wm$ z6~I`}0^8;h@HgO%$nA(*nu9(j{FbSVPv(hw+Q3O*HpWUD(%%Z|JohN$;NGUC6zyaU zzDOT^^@~l5z(Qd8F?ChZziMy)rqVy%j?LSzwb$s^yL>0``Zw$t#eSp|8w@+Pj9b^E zDlf*|EA=odXo{NX!)y9B$FFmK){k%dd%C^;MG-;y^9 zy@Q;W^xRN`xE5Q8P-1A;vbKLy5w*3>{J?x-@Rig-W9m$*{Sfoxn-6_1`rvO49M=HH zFLYQ9a_>0bF-_tv8ZcRw)5~{Fz9O^ln%qmSim3Uvcq4a_oB9!Qw7Y4~O?!b0?JtNv z$-=ybamFb%1-3K*hX=s9wS|RUaPBzsBRFvnbuZ|noTJcZc80+$^j6!U7u(lIrEI57 z{LhoDr>)@OAjkO~|ugkNm_oZF$ng2=pdvF6g0&BPXNhSGP6Q@K>P1Aylnu$e| zIAn3njlew%S$HNo0?Va)4wROoTLBi6{O#Kkxd#unh+h_dyc;y0V;dCS{KK-GtR2(0 zPM)Ir`QFv?PT!YQkmSSO2Q4|XWcaPlrQiU0rGq4gYn7}m`ozA}lFlBZ&~pnM4w=ax z6^{%WIr;vf*kOb=d~D^HsA<^F)1SVKJ!Vf$V3d8ubnVHr$HVA1_b1E=Ng)^04{Rhy zI_D~PVqfC=58wMjB4-xP9!Z7R#}>iE7HNsq@VSjaUwHQC@3S-pDGmE}k&%k61?PuL zSq`N=`b+7H$Z)4 zc-5LUs=CJJpQYzAo#Y+6M!#oox@a zpH1*zjIXxZtd9EiX(l#OtY*IyTNl$ic2I5a*k{y5;MzI^m~M|lZyKwn?cuvUg}1bc zO<@9cwvl5YoBHO~(FBeEqVJg(vu{cKPN* z)_xH1k^bms73ym!EWFa!qFBz`Uvz9~r(6TEDP;^|x1B}bv!w6%_V(1@b674yz)PMdkj)Nw1$>eEmCTC#^Vo)=HjXaJLbEe(*Qr0Q0|3{QS`stVkHL zr>423^mX_hFdQLBVJ;L0d{OCsh+!%1=jX!p*nn}A) zzjS$(&@~OX&*M6V7}gQG9U{CCcqFm8ew;w;KyuD|z;`SCOF;f3agE;@Syhut`Q{eo zlGrx|%;81mt}I~sMO77ZllT`6*rBbcqc5JJ#8}2YRP09Euy_3&S`+iU{Ytq7&;-h! z%@uf?jC`d}_Q`aVhebCE{yx6q@+yn!UtL?6uj7Nv`*Cc9CNcNyJ0Dv@CZT`nU+pUN zq)ftAsG0tb)dp0{ya$4tS^B$#los}A*^8RZx00VF!F_xH@i$7E$JIVrHx52<-+f